You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "blambov (via GitHub)" <gi...@apache.org> on 2023/04/07 10:58:26 UTC

[GitHub] [cassandra] blambov opened a new pull request, #2267: CASSANDRA-18398: Trie-indexed SSTable format

blambov opened a new pull request, #2267:
URL: https://github.com/apache/cassandra/pull/2267

   Implementation of the trie-indexed SSTable format (BTI) as described in CEP-25.
   
   Documentation in the included [BTIFormat.md](https://github.com/blambov/cassandra/blob/CASSANDRA-18398/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md).
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179509472


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ * <p>
+ * Uses IncrementalTrieWriter to build a trie of index section separators of the shortest possible length such that
+ * prevMax < separator <= nextMin.
+ */
+class RowIndexWriter implements AutoCloseable
+{
+    private final ClusteringComparator comparator;
+    private final IncrementalTrieWriter<IndexInfo> trie;
+    private ByteComparable prevMax = null;
+    private ByteComparable prevSep = null;
+
+    RowIndexWriter(ClusteringComparator comparator, DataOutputPlus out)
+    {
+        this.comparator = comparator;
+        this.trie = IncrementalTrieWriter.open(RowIndexReader.trieSerializer, out);
+    }
+
+    void reset()
+    {
+        prevMax = null;
+        prevSep = null;
+        trie.reset();
+    }
+
+    @Override
+    public void close()
+    {
+        trie.close();
+    }
+
+    void add(ClusteringPrefix<?> firstName, ClusteringPrefix<?> lastName, IndexInfo info) throws IOException
+    {
+        assert info.openDeletion != null;
+        ByteComparable sep = prevMax == null
+                             ? ByteComparable.EMPTY
+                             : ByteComparable.separatorGt(prevMax, comparator.asByteComparable(firstName));
+        trie.add(sep, info);
+        prevSep = sep;
+        prevMax = comparator.asByteComparable(lastName);
+    }
+
+    public long complete(long endPos) throws IOException
+    {
+        // Add a separator after the last section, so that greater inputs can be quickly rejected.
+        // To maximize its efficiency we add it with the length of the last added separator.
+        int i = 0;
+        ByteSource max = prevMax.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        ByteSource sep = prevSep.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        int c;
+        while ((c = max.next()) == sep.next() && c != ByteSource.END_OF_STREAM)
+            ++i;
+        assert c != ByteSource.END_OF_STREAM : "Corrupted row order, max=" + prevMax;
+
+        trie.add(nudge(prevMax, i), new IndexInfo(endPos, DeletionTime.LIVE));
+
+        return trie.complete();
+    }
+
+    /**
+     * Produces a source that is slightly greater than argument with length at least nudgeAt.
+     */
+    private ByteComparable nudge(ByteComparable value, int nudgeAt)
+    {
+        return version -> new ByteSource()
+        {
+            private final ByteSource v = value.asComparableBytes(version);
+            private int cur = 0;
+
+            @Override
+            public int next()
+            {
+                int b = ByteSource.END_OF_STREAM;
+                if (cur <= nudgeAt)
+                {
+                    b = v.next();
+                    if (cur == nudgeAt)
+                    {
+                        if (b < 255)

Review Comment:
   nit: 255 is canonically the max value for an unsigned byte I guess, but making it a constant somewhere wouldn't be terrible. Not super important...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180805146


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)

Review Comment:
   Is it possible for bytes to be anything other than `0` or `> 7`, given we use `nonZeroSize()`? Is it just equivalent to this?
   ```
   return bytes > 7 && buf.get(payloadPosition()) == (byte) hashBits;
   ```
   Tests seem to pass, in any case...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179750209


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179692143


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            IndexInfo payload = node.payload();
+            int bytes = 0;
+            int hasOpenMarker = 0;
+            if (payload != null)
+            {
+                bytes = SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    hasOpenMarker = FLAG_OPEN_MARKER;
+            }
+            type.serialize(dest, node, bytes | hasOpenMarker, nodePosition);
+            if (payload != null)
+            {
+                SizedInts.write(dest, payload.offset, bytes);
+
+                if (hasOpenMarker != 0)

Review Comment:
   nit: Would it be safer for this to be `hasOpenMarker == FLAG_OPEN_MARKER`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181881751


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.

Review Comment:
   ```suggestion
        * Called by {@link #getRowIndexEntry} above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181891098


##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ *
+ * The difference between this and PartitionIndexIterator is that this only uses information present in the index file

Review Comment:
   nit: "PartitionIterator" or "PartitionIndexIterator"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182998610


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");

Review Comment:
   ...or perhaps just the SSTable descriptor `toString()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183000571


##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ * <p>
+ * The difference between this and {@Link PartitionIterator} is that this only uses information present in the index file
+ * and does not try to read keys of the data file (for the trie index format), thus {@link #key()} can be null.
+ * <p>
+ * Starts advanced to a position, {@link #advance()} is to be used to go to next, and iteration completes when
+ * {@link #dataPosition()} == -1.
+ */
+public interface ScrubPartitionIterator extends Closeable
+{
+    /**
+     * Serialized partition key or {@code null} if the iterator reached the end of the index or if the key may not
+     * be fully retrieved from the index file.
+     * @return

Review Comment:
   nit: Remove the `@return`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183083400


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();

Review Comment:
   ```suggestion
               ClusteringBound<?> start = slice.start();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181875303


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant

Review Comment:
   ```suggestion
        * Whether to filter out data before {@link #first}. Needed for sources of data in a compaction, where the relevant
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181912317


##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.

Review Comment:
   Although the naming `TrieIndexEntry` is appropriate, given it could be partition or row.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173529081


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public boolean hasIsTransient()
+        {
+            return version.compareTo("ca") >= 0;
+        }
+
+        @Override
+        public boolean hasMetadataChecksum()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasOldBfFormat()
+        {
+            return hasOldBfFormat;
+        }
+
+        @Override
+        public boolean hasAccurateMinMax()
+        {
+            return hasAccurateMinMax;
+        }
+
+        public boolean hasLegacyMinMax()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean hasOriginatingHostId()
+        {
+            return hasOriginatingHostId;
+        }
+
+        @Override
+        public boolean hasImprovedMinMax() {
+            return hasImprovedMinMax;
+        }
+
+        @Override
+        public boolean hasPartitionLevelDeletionsPresenceMarker()
+        {
+            return hasPartitionLevelDeletionsPresenceMarker;
+        }
+
+        @Override
+        public boolean hasKeyRange()
+        {
+            return hasKeyRange;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);

Review Comment:
   I don't understand the comment.
   
   This follows the pattern in existing format handling, where we support older and current major versions (i.e. `charAt(0)`), and we also accept future minor version changes for all supported majors.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167181166


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReverseIterator.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.PrintStream;
+
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.ReverseValueIterator;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reverse iterator over the row index. Needed to get previous index blocks for reverse iteration.
+ */
+class RowIndexReverseIterator extends ReverseValueIterator<RowIndexReverseIterator>
+{
+    private long currentNode = -1;
+
+    public RowIndexReverseIterator(FileHandle file, long root, ByteComparable start, ByteComparable end)
+    {
+        super(file.instantiateRebufferer(null), root, start, end, true);
+    }
+
+    public RowIndexReverseIterator(FileHandle file, TrieIndexEntry entry, ByteComparable end)
+    {
+        this(file, entry.indexTrieRoot, ByteComparable.EMPTY, end);
+    }
+
+    /**
+     * This method must be async-read-safe.
+     */
+    public IndexInfo nextIndexInfo()
+    {
+        if (currentNode == -1)
+        {
+            currentNode = nextPayloadedNode();
+            if (currentNode == -1)
+                return null;
+        }
+
+        go(currentNode);
+        IndexInfo info = RowIndexReader.readPayload(buf, payloadPosition(), payloadFlags());
+
+        currentNode = -1;
+        return info;
+    }
+
+    public void dumpTrie(PrintStream out)
+    {
+        dumpTrie(out, (buf, ppos, bits) -> {
+            IndexInfo ii = RowIndexReader.readPayload(buf, ppos, bits);

Review Comment:
   nit: I guess `readPayload()` could return null here and cause problems



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167172430


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167089202


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+
+                    if (online)

Review Comment:
   nit: `online` is always true



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1567177016

   Committed


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164686408


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        Unfiltered next = null;
+        boolean foundLessThan;
+        long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();
+            currentPosition = file.getFilePointer();
+            foundLessThan = false;
+            // This is a copy of handlePreSliceData which also checks currentPosition < stopPosition.
+            // Not extracted to method as we need both marker and currentPosition.
+            if (filterStart)
+            {
+                while (currentPosition < stopPosition && deserializer.hasNext() && deserializer.compareNextTo(start) <= 0)
+                {
+                    if (deserializer.nextIsRow())
+                        deserializer.skipNext();
+                    else
+                        updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                    currentPosition = file.getFilePointer();
+                    foundLessThan = true;
+                }
+            }
+
+            // We've reached the beginning of our queried slice. If we have an open marker
+            // we should return that at the end of the slice to close the deletion.
+            if (openMarker != null)
+                blockOpenMarker = new RangeTombstoneBoundMarker(start, openMarker);
+
+
+            // Now deserialize everything until we reach our requested end (if we have one)
+            // See SSTableIterator.ForwardRead.computeNext() for why this is a strict inequality below: this is the same
+            // reasoning here.
+            while (currentPosition < stopPosition && deserializer.hasNext()
+                   && (!filterEnd || deserializer.compareNextTo(slice.end()) < 0))
+            {
+                rowOffsets.push(currentPosition);
+                if (deserializer.nextIsRow())
+                    deserializer.skipNext();
+                else
+                    updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                currentPosition = file.getFilePointer();
+            }
+
+            // If we have an open marker, we should output that first, unless end is not being filtered
+            // (i.e. it's either top (where a marker can't be open) or we placed that marker during previous block).
+            if (openMarker != null && filterEnd)
+            {
+                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                blockCloseMarker = new RangeTombstoneBoundMarker(slice.end(), openMarker);
+                openMarker = null;
+            }
+        }
+    }
+
+    private class ReverseIndexedReader extends ReverseReader
+    {
+        private RowIndexReverseIterator indexReader;
+        final TrieIndexEntry indexEntry;
+        long basePosition;

Review Comment:
   ```suggestion
           final long basePosition;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162099203


##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries
+
+    @Override
+    public int maxBytesInPage()
+    {
+        return buffer.capacity();
+    }
+
+    @Override
+    public void padToPageBoundary() throws IOException
+    {
+        if (buffer.position() == 0)
+            return;
+
+        int padLength = buffer.remaining();
+
+        // Flush as much as we have
+        doFlush(0);
+        // But pretend we had a whole chunk
+        bufferOffset += padLength;
+        lastFlushOffset += padLength;
+    }
+
+    @Override
+    public int bytesLeftInPage()
+    {
+        return buffer.remaining();
+    }
+
+    @Override
+    public long paddedPosition()
+    {
+        return position() + (buffer.position() == 0 ? 0 : buffer.remaining());
+    }
+

Review Comment:
   nit: Didn't find any test coverage for these new padding-related methods, but my search was probably too shallow...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167082925


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public class BtiTableScanner implements ISSTableScanner
+{
+    private final AtomicBoolean isClosed = new AtomicBoolean(false);
+    protected final RandomAccessReader dfile;
+    public final BtiTableReader sstable;
+
+    private final Iterator<AbstractBounds<PartitionPosition>> rangeIterator;
+
+    private final ColumnFilter columns;
+    private final DataRange dataRange;
+    private final SSTableReadsListener listener;
+    private long startScan = -1;
+    private long bytesScanned = 0;
+
+    protected CloseableIterator<UnfilteredRowIterator> iterator;
+
+    // Full scan of the sstables
+    public static ISSTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        assert sstable != null;
+
+        this.dfile = sstable.openDataReader();
+        this.sstable = sstable;
+        this.columns = columns;
+        this.dataRange = dataRange;
+        this.rangeIterator = rangeIterator;
+        this.listener = listener;
+    }
+
+    public static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(tokenRanges.size());
+        for (Range<Token> range : Range.normalize(tokenRanges))
+            addRange(sstable, Range.makeRowRange(range), boundsList);
+        return boundsList;
+    }
+
+    static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, DataRange dataRange)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(2);
+        addRange(sstable, dataRange.keyRange(), boundsList);
+        return boundsList;
+    }
+
+    static AbstractBounds<PartitionPosition> fullRange(SSTableReader sstable)
+    {
+        return new Bounds<>(sstable.first, sstable.last);
+    }
+
+    private static void addRange(SSTableReader sstable, AbstractBounds<PartitionPosition> requested, List<AbstractBounds<PartitionPosition>> boundsList)
+    {
+        if (requested instanceof Range && ((Range<?>) requested).isWrapAround())
+        {
+            if (requested.right.compareTo(sstable.first) >= 0)
+            {
+                // since we wrap, we must contain the whole sstable prior to stopKey()
+                Boundary<PartitionPosition> left = new Boundary<>(sstable.first, true);
+                Boundary<PartitionPosition> right;
+                right = requested.rightBoundary();
+                right = minRight(right, sstable.last, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+            if (requested.left.compareTo(sstable.last) <= 0)
+            {
+                // since we wrap, we must contain the whole sstable after dataRange.startKey()
+                Boundary<PartitionPosition> right = new Boundary<>(sstable.last, true);
+                Boundary<PartitionPosition> left;
+                left = requested.leftBoundary();
+                left = maxLeft(left, sstable.first, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+        }
+        else
+        {
+            assert !AbstractBounds.strictlyWrapsAround(requested.left, requested.right);
+            Boundary<PartitionPosition> left, right;
+            left = requested.leftBoundary();
+            right = requested.rightBoundary();
+            left = maxLeft(left, sstable.first, true);
+            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
+            right = requested.right.isMinimum() ? new Boundary<>(sstable.last, true)
+                                                : minRight(right, sstable.last, true);
+            if (!isEmpty(left, right))
+                boundsList.add(AbstractBounds.bounds(left, right));
+        }
+    }
+
+    public void close()
+    {
+        try
+        {
+            if (isClosed.compareAndSet(false, true))
+            {
+                FileUtils.close(dfile);
+                if (iterator != null)
+                    iterator.close();
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    public long getBytesScanned()
+    {
+        return bytesScanned;
+    }
+
+    @Override
+    public long getLengthInBytes()
+    {
+        return sstable.uncompressedLength();
+    }
+
+
+    public long getCompressedLengthInBytes()
+    {
+        return sstable.onDiskLength();
+    }
+
+    @Override
+    public long getCurrentPosition()
+    {
+        return dfile.getFilePointer();
+    }
+
+    @Override
+    public Set<SSTableReader> getBackingSSTables()
+    {
+        return ImmutableSet.of(sstable);
+    }
+
+    public int level()
+    {
+        return sstable.getSSTableLevel();
+    }
+
+    public TableMetadata metadata()
+    {
+        return sstable.metadata();
+    }
+
+    public boolean hasNext()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.hasNext();
+    }
+
+    public UnfilteredRowIterator next()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.next();
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    private CloseableIterator<UnfilteredRowIterator> createIterator()
+    {
+        this.listener.onScanningStarted(sstable);
+        return new KeyScanningIterator();
+    }
+
+    protected class KeyScanningIterator extends AbstractIterator<UnfilteredRowIterator> implements CloseableIterator<UnfilteredRowIterator>
+    {
+        private DecoratedKey currentKey;
+        private TrieIndexEntry currentEntry;
+        private PartitionIterator iterator;
+        private LazilyInitializedUnfilteredRowIterator currentRowIterator;

Review Comment:
   nit: `currentRowIterator` could just be a local variable? ...or just removed entirely if `computeNext()` returns the new `LazilyInitializedUnfilteredRowIterator` inline?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167155629


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings("resource")
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight) throws IOException

Review Comment:
   nit: unnecessary `throws`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167075760


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings("resource")

Review Comment:
   ```suggestion
               @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167193288


##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;

Review Comment:
   nit: `node` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167288178


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException
+     */
+    public static boolean equalsWithShortLength(FileDataInput in, ByteBuffer toMatch) throws IOException

Review Comment:
   nit: Untested codepath?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171597824


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java:
##########
@@ -345,11 +339,15 @@ protected UnfilteredRowIterator computeNext()
 
                 /*
                  * For a given partition key, we want to avoid hitting the data
-                 * file unless we're explicitely asked to. This is important
+                 * file unless we're explicitly asked to. This is important
                  * for PartitionRangeReadCommand#checkCacheFilter.
                  */
                 return new LazilyInitializedUnfilteredRowIterator(currentKey)
                 {
+                    // Store currentEntry reference during object instantiation as later (during initialize) the
+                    // reference may point to a different entry.
+                    private final RowIndexEntry rowIndexEntry = currentEntry;

Review Comment:
   Test for the associated failure plus proper fix added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181876944


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);

Review Comment:
   nit: Untested code path?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180810410


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;

Review Comment:
   nit: Untested code path?



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Check for a prefix and find closest smaller branch.
+            Long indexPos = prefixAndNeighbours(key, Reader::getSpecificIndexPos);
+
+            if (indexPos != null && indexPos != NOT_FOUND)
+            {
+                ResultType res = acceptor.accept(indexPos, false, key);
+                if (res != null)
+                    return res;
+            }
+
+            // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+            // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+            // prefix match that is closer than max(lesserBranch).
+            if (lesserBranch == NONE)
+                return null;
+            goMax(lesserBranch);
+            indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;

Review Comment:
   nit: Untested code path?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180874713


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,

Review Comment:
   nit: `@VisibleForTesting` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179713120


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    protected TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy(() -> FBUtilities.immutableListWithFilteredNulls(iwriter, dataWriter));
+    }
+
+    private class TransactionalProxy extends SortedTableWriter<BtiFormatPartitionWriter>.TransactionalProxy
+    {
+        public TransactionalProxy(Supplier<ImmutableList<Transactional>> transactionals)
+        {
+            super(transactionals);
+        }
+
+        @Override
+        protected Throwable doPostCleanup(Throwable accumulate)
+        {
+            accumulate = Throwables.close(accumulate, partitionWriter);
+            accumulate = super.doPostCleanup(accumulate);
+            return accumulate;
+        }
+    }
+
+    /**
+     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
+     */
+    static class IndexWriter extends SortedTableWriter.AbstractIndexWriter
+    {
+        final SequentialWriter rowIndexWriter;
+        private final FileHandle.Builder rowIndexFHBuilder;
+        private final SequentialWriter partitionIndexWriter;
+        private final FileHandle.Builder partitionIndexFHBuilder;
+        private final PartitionIndexBuilder partitionIndex;
+        boolean partitionIndexCompleted = false;
+        private DataPosition riMark;
+        private DataPosition piMark;
+
+        IndexWriter(Builder b)
+        {
+            super(b);
+            rowIndexWriter = new SequentialWriter(descriptor.fileFor(Components.ROW_INDEX), b.getIOOptions().writerOptions);
+            rowIndexFHBuilder = IndexComponent.fileBuilder(Components.ROW_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndexWriter = new SequentialWriter(descriptor.fileFor(Components.PARTITION_INDEX), b.getIOOptions().writerOptions);
+            partitionIndexFHBuilder = IndexComponent.fileBuilder(Components.PARTITION_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndex = new PartitionIndexBuilder(partitionIndexWriter, partitionIndexFHBuilder);
+            // register listeners to be alerted when the data files are flushed
+            partitionIndexWriter.setPostFlushListener(() -> partitionIndex.markPartitionIndexSynced(partitionIndexWriter.getLastFlushOffset()));
+            rowIndexWriter.setPostFlushListener(() -> partitionIndex.markRowIndexSynced(rowIndexWriter.getLastFlushOffset()));
+            @SuppressWarnings("resource")

Review Comment:
   ```suggestion
               @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183005316


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();

Review Comment:
   nit: Brief message, given this could be used via standalone scrubber and therefore be user-facing?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1184160207


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ *   - calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   - delaying writing any part of a completed node until its branch size is above the page size
+ *   - laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   - adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ *
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ *
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * where lines denote page boundaries.
+ *
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+                child.filePos = write(child);
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+                || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    protected String dumpNode(Node<VALUE> node, long nodePosition)
+    {
+        StringBuilder res = new StringBuilder(String.format("At %,d(%x) type %s child count %s nodeSize %,d branchSize %,d %s%s%n",
+                                                            nodePosition, nodePosition,
+                                                            TrieNode.typeFor(node, nodePosition), node.childCount(), node.nodeSize, node.branchSize,
+                                                            node.hasOutOfPageChildren ? "C" : "",
+                                                            node.hasOutOfPageInBranch ? "B" : ""));
+        for (Node<VALUE> child : node.children)
+            res.append(String.format("Child %2x at %,d(%x) type %s child count %s size %s nodeSize %,d branchSize %,d %s%s%n",
+                                     child.transition & 0xFF,
+                                     child.filePos,
+                                     child.filePos,
+                                     child.children != null ? TrieNode.typeFor(child, child.filePos) : "n/a",
+                                     child.children != null ? child.childCount() : "n/a",
+                                     child.children != null ? serializer.sizeofNode(child, child.filePos) : "n/a",
+                                     child.nodeSize,
+                                     child.branchSize,
+                                     child.hasOutOfPageChildren ? "C" : "",
+                                     child.hasOutOfPageInBranch ? "B" : ""));
+
+        return res.toString();
+    }
+
+    @Override
+    public PartialTail makePartialRoot() throws IOException
+    {
+        // The expectation is that the partial tail will be in memory, so we don't bother with page-fitting.
+        // We could also send some completed children to disk, but that could make suboptimal layout choices, so we'd
+        // rather not. Just write anything not written yet to a buffer, from bottom to top, and we're done.
+        try (DataOutputBuffer buf = new DataOutputBuffer())
+        {
+            PTail tail = new PTail();
+            // Readers ask rebufferers for page-aligned positions, so make sure tail starts at one.
+            // "Padding" of the cutoff point may leave some unaddressable space in the constructed file view.
+            // Nothing will point to it, though, so that's fine.
+            tail.cutoff = dest.paddedPosition();
+            tail.count = count;
+            tail.root = writePartial(stack.getFirst(), buf, tail.cutoff);
+            tail.tail = buf.asNewBuffer();
+            return tail;
+        }
+    }
+
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                child.filePos = writePartial(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    static class Node<Value> extends IncrementalTrieWriterBase.BaseNode<Value, Node<Value>>
+    {
+        /**
+         * Currently calculated size of the branch below this node, not including the node itself.
+         * If hasOutOfPageInBranch is true, this may be underestimated as the size
+         * depends on the position the branch is written.
+         */
+        int branchSize = -1;
+        /**
+         * Currently calculated node size. If hasOutOfPageChildren is true, this may be underestimated as the size
+         * depends on the position the node is written.
+         */
+        int nodeSize = -1;
+
+        /**
+         * Whether there is an out-of-page, already written node in the branches below the immediate children of the
+         * node.
+         */
+        boolean hasOutOfPageInBranch = false;
+        /**
+         * Whether a child of the node is out of page, already written.
+         * Forced to true before being set to make sure maxPositionDelta performs its evaluation on non-completed
+         * nodes for makePartialRoot.
+         */
+        boolean hasOutOfPageChildren = true;
+
+        Node(int transition)
+        {
+            super(transition);
+        }
+
+        @Override
+        Node<Value> newNode(byte transition)
+        {
+            return new Node<>(transition & 0xFF);
+        }
+
+        public long serializedPositionDelta(int i, long nodePosition)
+        {
+            assert (children.get(i).filePos != -1);
+            return children.get(i).filePos - nodePosition;
+        }
+
+        /**
+         * The max delta is the delta with either:
+         * - the position where the first child not-yet-placed child will be laid out.
+         * - the position of the furthest child that is already placed.
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1195177668


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1202703695


##########
conf/cassandra.yaml:
##########
@@ -968,8 +968,8 @@ snapshot_links_per_second: 0
 # loaded using the service loader mechanism. In this section, one can select
 # the format for created sstables and pass additional parameters for the formats
 # available on the classpath.
-# The default format is BIG, the legacy SSTable format in use since Cassandra 3.0.
-# Cassandra versions 5.0 and later also support the trie-indexed BTI format,
+# The default format is "big", the legacy SSTable format in use since Cassandra 3.0.
+# Cassandra versions 5.0 and later also support the trie-indexed "bti" format,

Review Comment:
   LGTM, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185512818


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException

Review Comment:
   nit: remove `InterruptedException`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162272763


##########
src/java/org/apache/cassandra/db/DeletionTime.java:
##########
@@ -191,4 +201,4 @@ public long serializedSize(DeletionTime delTime)
                  + TypeSizes.sizeof(delTime.markedForDeleteAt());
         }
     }
-}
+}

Review Comment:
   seems no necessary to remove and add a new line.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format

Review Comment:
   I think these descriptions for DSE can be removed ,although dse and apache cassandra may have some in common, but I think developers may do not really know the version or sstable version for DSE. 
   some descriptions for apache cassandra sstable may be more suitable。



##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries

Review Comment:
   What does line 340 means ? a comment for method "maxBytesInPage()" ? 



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba

Review Comment:
   what does this comment LABS mean ?



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format

Review Comment:
   I think as this sstable format may be used in dse for long time ,so these comments are left, in my mind, we may need to add comments to identify this file format, as this is the first time we introduce this sstable file format to apache cassandra, so we may do not need to describle the differences betweetn dse versions for Trie-indexed sstable, only need to add some comments about the  reason this file format has now evolved to the current committed version.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format

Review Comment:
   as these versions may be used int the construct function of BtiVersion, and as cassandra do not have these sstable versions as dse have ,so I think we need to modify the code here to adapt to the existing file format(version) of apache cassandra



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;

Review Comment:
   I think rowStart can be move to line 95 in the while loop 



##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries

Review Comment:
   What does line 340 means ? a comment for method "maxBytesInPage()" ? 



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (IOException e)
+        {
+            outputHandler.warn("Index is unreadable.");

Review Comment:
   I think the message is not detailed enough, and the exception stack can be added.



##########
conf/cassandra.yaml:
##########
@@ -1924,8 +1924,14 @@ drop_compact_storage_enabled: false
 # which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
 # which will be used to recognize the format type - in particular that name will be used in sstable file names and in
 # stream headers so the name has to be the same for the same format across all the nodes in the cluster.
+# The first entry in this list is the format that will be used for newly-created SSTables. The other formats given
+# will be used to read any SSTables present in the data directories or streamed.
 sstable_formats:
   - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
     parameters:
       id: 0
       name: big
+  - class_name: org.apache.cassandra.io.sstable.format.bti.BtiFormat
+    parameters:
+      id: 1
+      name: bti

Review Comment:
   This seems introduce in CASSANDRA-17056,  aggree with "Hard to know without actually writing the code" ,  and I think at the very least we should add more detailed comments to explain the useage of "sstable_formats" .
   I think it is not enough for users to configure this options if they do not read the code, "what does the id mean ? " , "Can I set the id to 100? ", "what does name mean? "



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164440372


##########
src/java/org/apache/cassandra/io/sstable/format/big/BigTableScanner.java:
##########
@@ -345,11 +339,15 @@ protected UnfilteredRowIterator computeNext()
 
                 /*
                  * For a given partition key, we want to avoid hitting the data
-                 * file unless we're explicitely asked to. This is important
+                 * file unless we're explicitly asked to. This is important
                  * for PartitionRangeReadCommand#checkCacheFilter.
                  */
                 return new LazilyInitializedUnfilteredRowIterator(currentKey)
                 {
+                    // Store currentEntry reference during object instantiation as later (during initialize) the
+                    // reference may point to a different entry.
+                    private final RowIndexEntry rowIndexEntry = currentEntry;

Review Comment:
   Was there are test failing somewhere around this, or is it just something we noticed offhand?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162079953


##########
src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java:
##########
@@ -105,4 +105,4 @@ public void close()
         if (iterator != null)
             iterator.close();
     }
-}
+}

Review Comment:
   nit: necessary to remove? I have no idea if we have a style-guide thing for EoF newlines :D



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167157130


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);

Review Comment:
   unused logger



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167075000


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")

Review Comment:
   ```suggestion
                   @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```
   nit: We've started doing this in a few places to remove noise for people using IDEA, FWIW



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185290350


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException
+        {
+            nodePosition = nodePosition + node.branchSize;
+            assert dest.position() == nodePosition
+                    : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+            serializer.write(dest, node, nodePosition);
+
+            assert dest.position() == nodePosition + node.nodeSize
+                   || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                    : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+
+            node.filePos = nodePosition;
+        }
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    class WritePartialRecursion extends Recursion<Node<VALUE>>
+    {
+        final DataOutputPlus dest;
+        final long baseOffset;
+        final long startPosition;
+        final List<Node<VALUE>> childrenToClear;
+
+        WritePartialRecursion(Node<VALUE> node, WritePartialRecursion parent)
+        {
+            super(node, node.children.iterator(), parent);
+            this.dest = parent.dest;
+            this.baseOffset = parent.baseOffset;
+            this.startPosition = dest.position() + baseOffset;
+            childrenToClear = new ArrayList<>();
+        }
+
+        WritePartialRecursion(Node<VALUE> node, DataOutputPlus dest, long baseOffset)
+        {
+            super(node, node.children.iterator(), null);
+            this.dest = dest;
+            this.baseOffset = baseOffset;
+            this.startPosition = dest.position() + baseOffset;
+            childrenToClear = new ArrayList<>();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                return new WritePartialRecursion(child, this);
+            }
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException

Review Comment:
   nit: Seems like the duplication is intended, so maybe throw a `@SuppressWarnings("DuplicatedCode")` on there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1552816255

   I believe the above includes all currently requested changes. Additional work to be done in:
   - CASSANDRA-18441: changing the format selection mechanism
   - CASSANDRA-18533: moving format-specific settings into the format configuration
   - CASSANDRA-18534: per-table configuration
   - CASSANDRA-18535: index error-detection
   - CASSANDRA-18536: index encryption


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178360738


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //

Review Comment:
   nit: We already say all of this in the class JavaDoc for `Version`. Might as well remove it from here and `BigVersion`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179534244


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>

Review Comment:
   nit: Throw a `@NotThreadSafe` on the class?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179491343


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179753193


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1174735353


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public boolean hasIsTransient()
+        {
+            return version.compareTo("ca") >= 0;
+        }
+
+        @Override
+        public boolean hasMetadataChecksum()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasOldBfFormat()
+        {
+            return hasOldBfFormat;
+        }
+
+        @Override
+        public boolean hasAccurateMinMax()
+        {
+            return hasAccurateMinMax;
+        }
+
+        public boolean hasLegacyMinMax()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean hasOriginatingHostId()
+        {
+            return hasOriginatingHostId;
+        }
+
+        @Override
+        public boolean hasImprovedMinMax() {
+            return hasImprovedMinMax;
+        }
+
+        @Override
+        public boolean hasPartitionLevelDeletionsPresenceMarker()
+        {
+            return hasPartitionLevelDeletionsPresenceMarker;
+        }
+
+        @Override
+        public boolean hasKeyRange()
+        {
+            return hasKeyRange;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167166246


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public class BtiTableScanner implements ISSTableScanner
+{
+    private final AtomicBoolean isClosed = new AtomicBoolean(false);
+    protected final RandomAccessReader dfile;
+    public final BtiTableReader sstable;
+
+    private final Iterator<AbstractBounds<PartitionPosition>> rangeIterator;
+
+    private final ColumnFilter columns;
+    private final DataRange dataRange;
+    private final SSTableReadsListener listener;
+    private long startScan = -1;
+    private long bytesScanned = 0;
+
+    protected CloseableIterator<UnfilteredRowIterator> iterator;
+
+    // Full scan of the sstables
+    public static ISSTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        assert sstable != null;
+
+        this.dfile = sstable.openDataReader();
+        this.sstable = sstable;
+        this.columns = columns;
+        this.dataRange = dataRange;
+        this.rangeIterator = rangeIterator;
+        this.listener = listener;
+    }
+
+    public static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(tokenRanges.size());
+        for (Range<Token> range : Range.normalize(tokenRanges))
+            addRange(sstable, Range.makeRowRange(range), boundsList);
+        return boundsList;
+    }
+
+    static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, DataRange dataRange)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(2);
+        addRange(sstable, dataRange.keyRange(), boundsList);
+        return boundsList;
+    }
+
+    static AbstractBounds<PartitionPosition> fullRange(SSTableReader sstable)
+    {
+        return new Bounds<>(sstable.first, sstable.last);
+    }
+
+    private static void addRange(SSTableReader sstable, AbstractBounds<PartitionPosition> requested, List<AbstractBounds<PartitionPosition>> boundsList)
+    {
+        if (requested instanceof Range && ((Range<?>) requested).isWrapAround())
+        {
+            if (requested.right.compareTo(sstable.first) >= 0)
+            {
+                // since we wrap, we must contain the whole sstable prior to stopKey()
+                Boundary<PartitionPosition> left = new Boundary<>(sstable.first, true);
+                Boundary<PartitionPosition> right;
+                right = requested.rightBoundary();
+                right = minRight(right, sstable.last, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+            if (requested.left.compareTo(sstable.last) <= 0)
+            {
+                // since we wrap, we must contain the whole sstable after dataRange.startKey()
+                Boundary<PartitionPosition> right = new Boundary<>(sstable.last, true);
+                Boundary<PartitionPosition> left;
+                left = requested.leftBoundary();
+                left = maxLeft(left, sstable.first, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+        }
+        else
+        {
+            assert !AbstractBounds.strictlyWrapsAround(requested.left, requested.right);
+            Boundary<PartitionPosition> left, right;
+            left = requested.leftBoundary();
+            right = requested.rightBoundary();
+            left = maxLeft(left, sstable.first, true);
+            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
+            right = requested.right.isMinimum() ? new Boundary<>(sstable.last, true)
+                                                : minRight(right, sstable.last, true);
+            if (!isEmpty(left, right))
+                boundsList.add(AbstractBounds.bounds(left, right));
+        }
+    }
+
+    public void close()
+    {
+        try
+        {
+            if (isClosed.compareAndSet(false, true))
+            {
+                FileUtils.close(dfile);
+                if (iterator != null)
+                    iterator.close();
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    public long getBytesScanned()
+    {
+        return bytesScanned;
+    }
+
+    @Override
+    public long getLengthInBytes()
+    {
+        return sstable.uncompressedLength();
+    }
+
+
+    public long getCompressedLengthInBytes()
+    {
+        return sstable.onDiskLength();
+    }
+
+    @Override
+    public long getCurrentPosition()
+    {
+        return dfile.getFilePointer();
+    }
+
+    @Override
+    public Set<SSTableReader> getBackingSSTables()
+    {
+        return ImmutableSet.of(sstable);
+    }
+
+    public int level()

Review Comment:
   nit: unused method



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1509245174

   We've already talked about this a bit, but my feedback for changes to the `o.a.c.io.tries` and `o.a.c.utils.concurrent` is mostly to make sure we keep things in sync w/ the SAI patch for string/literal indexes at https://github.com/apache/cassandra/commit/b45f47d7097b2c9fbf6abe1c10ae60543b0068cb.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178386056


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file

Review Comment:
   Should we use `partitionKeysSize()` for the keys in the data file? I'm probably just a little confused at a high level w/ this calculation overall...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180805146


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)

Review Comment:
   Is it possible for bytes to be anything other than `0` or `> 7`, given we use `nonZeroSize()`? Is it just equivalent to this?
   
   ```
   return bytes > 7 && buf.get(payloadPosition()) == (byte) hashBits;
   ```
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180769892


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;

Review Comment:
   Aside: So `size` is the number of bytes in the payload, but where does the `7` come from? (The hash is just one byte?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183033467


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;
+                    }
+                }
+            }
+
+            String keyName = key == null ? "(unreadable key)" : keyString(key);
+            outputHandler.debug("partition %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSizeFromIndex));
+
+            try
+            {
+                if (key == null)
+                    throw new IOError(new IOException("Unable to read partition key from data file", keyReadError));
+
+                if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
+                {
+                    throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
+                                                                    //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));

Review Comment:
   nit: Remove the commented out bit...or reintroduce it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178584091


##########
src/java/org/apache/cassandra/io/sstable/format/SSTableScanner.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public abstract class SSTableScanner<S extends SSTableReader,
+                                    E extends AbstractRowIndexEntry,

Review Comment:
   code format ? do we need one more space here to make S and E align



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1184153167


##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+@NotThreadSafe
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        final long node;
+        final int limit;
+        final IterationPosition prev;
+        int childIndex;
+
+        public IterationPosition(long node, int childIndex, int limit, IterationPosition prev)
+        {
+            super();
+            this.node = node;
+            this.childIndex = childIndex;
+            this.limit = limit;
+            this.prev = prev;
+        }
+    }
+
+    protected ReverseValueIterator(Rebufferer source, long root)

Review Comment:
   nit: Anything use this ctor?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162884362


##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries
+
+    @Override
+    public int maxBytesInPage()
+    {
+        return buffer.capacity();
+    }
+
+    @Override
+    public void padToPageBoundary() throws IOException
+    {
+        if (buffer.position() == 0)
+            return;
+
+        int padLength = buffer.remaining();
+
+        // Flush as much as we have
+        doFlush(0);
+        // But pretend we had a whole chunk
+        bufferOffset += padLength;
+        lastFlushOffset += padLength;
+    }
+
+    @Override
+    public int bytesLeftInPage()
+    {
+        return buffer.remaining();
+    }
+
+    @Override
+    public long paddedPosition()
+    {
+        return position() + (buffer.position() == 0 ? 0 : buffer.remaining());
+    }
+

Review Comment:
   I would be surprised if there are. The related functionality is not ported.
   
   Removed the methods, they are currently unused.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180770261


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Check for a prefix and find closest smaller branch.
+            Long indexPos = prefixAndNeighbours(key, Reader::getSpecificIndexPos);
+
+            if (indexPos != null && indexPos != NOT_FOUND)
+            {
+                ResultType res = acceptor.accept(indexPos, false, key);
+                if (res != null)
+                    return res;
+            }
+
+            // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+            // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+            // prefix match that is closer than max(lesserBranch).
+            if (lesserBranch == NONE)
+                return null;
+            goMax(lesserBranch);
+            indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public Long getSpecificIndexPos(int pos, int bits)
+        {
+            return getIndexPos(buf, pos, bits);
+        }
+
+        public long getCurrentIndexPos()
+        {
+            return getIndexPos(buf, payloadPosition(), payloadFlags());
+        }
+
+        public long getLastIndexPosition()
+        {
+            goMax(root);
+            return getCurrentIndexPos();
+        }
+
+        /**
+         * To be used only in analysis.
+         */
+        @SuppressWarnings("unused")
+        protected int payloadSize()
+        {
+            int bytes = payloadFlags();
+            return bytes > 7 ? bytes - 6 : bytes;
+        }
+    }
+
+    /**
+     * Iterator of index positions covered between two keys. Since we store prefixes only, the first and last returned
+     * values can be outside the span (and inclusiveness is not given as we cannot verify it).
+     */
+    public static class IndexPosIterator extends ValueIterator<IndexPosIterator>
+    {
+        static final long INVALID = -1;
+        long pos = INVALID;
+
+        /**
+         * @param index PartitionIndex to use for the iteration.
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181875545


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point

Review Comment:
   ```suggestion
        * Whether to filter out data after {@link #last}. Early-open sstables may contain data beyond the switch point
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181890827


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds

Review Comment:
   nit: "PartitionIterator" or "PartitionIndexIterator"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181907295


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }
+
+        return PartitionIterator.create(partitionIndex,
+                                        metadata().partitioner,
+                                        rowIndexFile,
+                                        dfile,
+                                        left, inclusiveLeft ? -1 : 0,
+                                        right, inclusiveRight ? 0 : -1);
+    }
+
+    @Override
+    public PartitionIterator keyReader() throws IOException
+    {
+        return PartitionIterator.create(partitionIndex, metadata().partitioner, rowIndexFile, dfile);
+    }
+
+    @Override
+    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
+    {
+        // BTI does not support key sampling as it would involve walking the index or data file.
+        // Validator has an alternate solution for empty key sample lists.
+        return Collections.emptyList();
+    }
+
+    @Override
+    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
+    {
+        // Estimate the number of partitions by calculating the bytes of the sstable that are covered by the specified
+        // ranges and using the mean partition size to obtain a number of partitions from that.
+        long selectedDataSize = 0;
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            PartitionPosition left = range.left.minKeyBound();
+            if (left.compareTo(first) <= 0)
+                left = null;
+            else if (left.compareTo(last) > 0)
+                continue;   // no intersection
+
+            PartitionPosition right = range.right.minKeyBound();
+            if (range.right.isMinimum() || right.compareTo(last) >= 0)
+                right = null;
+            else if (right.compareTo(first) < 0)
+                continue;   // no intersection
+
+            if (left == null && right == null)
+                return partitionIndex.size();   // sstable is fully covered, return full partition count to avoid rounding errors
+
+            if (left == null && filterFirst())
+                left = first;
+            if (right == null && filterLast())
+                right = last;
+
+            long startPos = left != null ? getPosition(left, GE) : 0;
+            long endPos = right != null ? getPosition(right, GE) : uncompressedLength();
+            selectedDataSize += endPos - startPos;
+        }
+        return Math.round(selectedDataSize / sstableMetadata.estimatedPartitionSize.rawMean());
+    }
+
+
+    @Override
+    public UnfilteredRowIterator rowIterator(DecoratedKey key,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed,
+                                             SSTableReadsListener listener)
+    {
+        return rowIterator(null, key, getExactPosition(key, listener, true), slices, selectedColumns, reversed);
+    }
+
+    public UnfilteredRowIterator rowIterator(FileDataInput dataFileInput,
+                                             DecoratedKey key,
+                                             TrieIndexEntry indexEntry,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed)
+    {
+        if (indexEntry == null)
+            return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
+
+        if (reversed)
+            return new SSTableReversedIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+        else
+            return new SSTableIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+    }
+
+    @Override
+    public ISSTableScanner getScanner()
+    {
+        return BtiTableScanner.getScanner(this);
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Collection<Range<Token>> ranges)
+    {
+        if (ranges != null)
+            return BtiTableScanner.getScanner(this, ranges);
+        else
+            return getScanner();
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return BtiTableScanner.getScanner(this, rangeIterator);
+    }
+
+    @VisibleForTesting
+    @Override
+    public BtiTableReader cloneAndReplace(IFilter filter)
+    {
+        return unbuildTo(new Builder(descriptor).setFilter(filter), true).build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public BtiTableReader cloneWithRestoredStart(DecoratedKey restoredStart)
+    {
+        return runWithLock(ignored -> cloneAndReplace(restoredStart, OpenReason.NORMAL));
+    }
+
+    @Override
+    public BtiTableReader cloneWithNewStart(DecoratedKey newStart)
+    {
+        return runWithLock(d -> {
+            assert openReason != OpenReason.EARLY;

Review Comment:
   nit: Tack on a brief message?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162091485


##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries
+
+    @Override
+    public int maxBytesInPage()
+    {
+        return buffer.capacity();
+    }
+
+    @Override
+    public void padToPageBoundary() throws IOException

Review Comment:
   nit:
   ```suggestion
       public void padToPageBoundary()
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1503450660

   I think this applies all requested changes from the SAI branch, except removing `PageAware.pad` which is an implementation I see as belonging to the class.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1195176307


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);

Review Comment:
   Added in `SSTableReaderTest.testGetPositionsBloomFilterStats`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1566733664

   I don't see anything related to the patch in the test failures, moving ahead with merge.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167198345


##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;
+        IterationPosition prev;

Review Comment:
   nit: `prev` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164684500


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableIterator.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ *  Unfiltered row iterator over a BTI SSTable.
+ */
+class SSTableIterator extends AbstractSSTableIterator<AbstractRowIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableIterator(BtiTableReader sstable,
+                           FileDataInput file,
+                           DecoratedKey key,
+                           AbstractRowIndexEntry indexEntry,
+                           Slices slices,
+                           ColumnFilter columns,
+                           FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(AbstractRowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ForwardIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ForwardReader(file, shouldCloseFile);
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return next;
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    public boolean isReverseOrder()
+    {
+        return false;
+    }
+
+    private class ForwardIndexedReader extends ForwardReader
+    {
+        private final RowIndexReader indexReader;
+        long basePosition;

Review Comment:
   nit: can be `final`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1163877511


##########
conf/cassandra.yaml:
##########
@@ -1924,8 +1924,14 @@ drop_compact_storage_enabled: false
 # which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
 # which will be used to recognize the format type - in particular that name will be used in sstable file names and in
 # stream headers so the name has to be the same for the same format across all the nodes in the cluster.
+# The first entry in this list is the format that will be used for newly-created SSTables. The other formats given
+# will be used to read any SSTables present in the data directories or streamed.
 sstable_formats:
   - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
     parameters:
       id: 0
       name: big
+  - class_name: org.apache.cassandra.io.sstable.format.bti.BtiFormat
+    parameters:
+      id: 1
+      name: bti

Review Comment:
   I'll fix the configuration in CASSANDRA-18441, I agree it is bad and was aimed too much to make everything configurable.
   
   (nit: @dcapwell that map is `ImmutableMap` which retains order, OTOH that's too bad that `DatabaseDescriptor` does not make it explicit and returns just a `Map`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167253745


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1170677847


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterSimple.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builder of on-disk tries. Takes sorted input.
+ * <p>
+ * Incremental building is done by maintaining a stack of nodes in progress which follows the path to reach the last
+ * added entry. When a new entry is needed, comparison with the previous can tell us how much of the parents stack
+ * remains the same. The rest of the stack is complete as no new entry can affect them due to the input sorting.
+ * The completed nodes can be written to disk and discarded, keeping only a pointer to their location in the file
+ * (this pointer will be discarded too when the parent node is completed). This ensures that a very limited amount of
+ * data is kept in memory at all times.
+ * <p>
+ * Note: This class is currently unused and stands only as form of documentation for {@link IncrementalTrieWriterPageAware}.

Review Comment:
   It's used in `WalkerTest`...perhaps we could just move it to the `test` folder?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1172479251


##########
src/java/org/apache/cassandra/io/tries/TrieNode.java:
##########
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SizedInts;
+
+/**
+ * Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as
+ * they are on disk without any serialization, and to enable the creation of such files.
+ * <p>
+ * The serialization methods take as argument a generic {@code SerializationNode} and provide a method {@code typeFor}
+ * for choosing a suitable type to represent it, which can then be used to calculate size and write the node.
+ * <p>
+ * To read a file containing trie nodes, one would use {@code at} to identify the node type and then the various
+ * read methods to retrieve the data. They all take a buffer (usually memory-mapped) containing the data, and a position
+ * in it that identifies the node.
+ * <p>
+ * These node types do not specify any treatment of payloads. They are only concerned with providing 4 bits of
+ * space for {@code payloadFlags}, and a way of calculating the position after the node. Users of this class by convention
+ * use non-zero payloadFlags to indicate a payload exists, write it (possibly in flag-dependent format) at serialization
+ * time after the node itself is written, and read it using the {@code payloadPosition} value.
+ * <p>
+ * To improve efficiency, multiple node types depending on the number of transitions are provided:
+ * -- payload only, which has no outgoing transitions
+ * -- single outgoing transition
+ * -- sparse, which provides a list of transition bytes with corresponding targets
+ * -- dense, where the transitions span a range of values and having the list (and the search in it) can be avoided
+ * <p>
+ * For each of the transition-carrying types we also have "in-page" versions where transition targets are the 4, 8 or 12
+ * lowest bits of the position within the same page. To save one further byte, the single in-page versions using 4 or 12
+ * bits cannot carry a payload.
+ * <p>
+ * This class is effectively an enumeration; abstract class permits instances to extend each other and reuse code.
+ */
+public abstract class TrieNode
+{
+    // Consumption (read) methods
+
+    /**
+     * Returns the type of node stored at this position. It can then be used to call the methods below.
+     */
+    public static TrieNode at(ByteBuffer src, int position)
+    {
+        return values[(src.get(position) >> 4) & 0xF];
+    }
+
+    /**
+     * Returns the 4 payload flag bits. Node types that cannot carry a payload return 0.
+     */
+    public int payloadFlags(ByteBuffer src, int position)
+    {
+        return src.get(position) & 0x0F;
+    }
+
+    /**
+     * Return the position just after the node, where the payload is usually stored.
+     */
+    abstract public int payloadPosition(ByteBuffer src, int position);
+
+    /**
+     * Returns search index for the given byte in the node. If exact match is present, this is >= 0, otherwise as in
+     * binary search.
+     */
+    abstract public int search(ByteBuffer src, int position, int transitionByte);       // returns as binarySearch
+
+    /**
+     * Returns the upper childIndex limit. Calling transition with values 0...transitionRange - 1 is valid.
+     */
+    abstract public int transitionRange(ByteBuffer src, int position);
+
+    /**
+     * Returns the byte value for this child index, or Integer.MAX_VALUE if there are no transitions with this index or
+     * higher to permit listing the children without needing to call transitionRange.
+     *
+     * @param childIndex must be >= 0, though it is allowed to pass a value greater than {@code transitionRange - 1}
+     */
+    abstract public int transitionByte(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns the delta between the position of this node and the position of the target of the specified transition.
+     * This is always a negative number. Dense nodes use 0 to specify "no transition".
+     *
+     * @param childIndex must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                   and behaviour of this method is undefined for values outside of that range
+     */
+    abstract long transitionDelta(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns position of node to transition to for the given search index. Argument must be positive. May return -1
+     * if a transition with that index does not exist (DENSE nodes).
+     * Position is the offset of the node within the ByteBuffer. positionLong is its global placement, which is the
+     * base for any offset calculations.
+     *
+     * @param positionLong although it seems to be obvious, this argument must be "real", that is, each child must have
+     *                     the calculated absolute position >= 0, otherwise the behaviour of this method is undefined
+     * @param childIndex   must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                     and behaviour of this method is undefined for values outside of that range
+     */
+    public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+    {
+        // note: this is not valid for dense nodes
+        return positionLong + transitionDelta(src, position, childIndex);
+    }
+
+    /**
+     * Returns the highest transition for this node, or -1 if none exist (PAYLOAD_ONLY nodes).
+     */
+    public long lastTransition(ByteBuffer src, int position, long positionLong)
+    {
+        return transition(src, position, positionLong, transitionRange(src, position) - 1);
+    }
+
+    /**
+     * Returns a transition that is higher than the index returned by {@code search}. This may not exist (if the
+     * argument was higher than the last transition byte), in which case this returns the given {@code defaultValue}.
+     */
+    abstract public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    /**
+     * Returns a transition that is lower than the index returned by {@code search}. Returns {@code defaultValue} for
+     * {@code searchIndex} equals 0 or -1 as lesser transition for those indexes does not exist.
+     */
+    abstract public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    // Construction (serialization) methods
+
+    /**
+     * Returns a node type that is suitable to store the node.
+     */
+    public static TrieNode typeFor(SerializationNode<?> node, long nodePosition)
+    {
+        int c = node.childCount();
+        if (c == 0)
+            return PAYLOAD_ONLY;
+
+        int bitsPerPointerIndex = 0;
+        long delta = node.maxPositionDelta(nodePosition);
+        assert delta < 0;
+        while (!singles[bitsPerPointerIndex].fits(-delta))
+            ++bitsPerPointerIndex;
+
+        if (c == 1)
+        {
+            if (node.payload() != null && singles[bitsPerPointerIndex].bytesPerPointer == FRACTIONAL_BYTES)
+                ++bitsPerPointerIndex; // next index will permit payload
+
+            return singles[bitsPerPointerIndex];
+        }
+
+        TrieNode sparse = sparses[bitsPerPointerIndex];
+        TrieNode dense = denses[bitsPerPointerIndex];
+        return (sparse.sizeofNode(node) < dense.sizeofNode(node)) ? sparse : dense;
+    }
+
+    /**
+     * Returns the size needed to serialize this node.
+     */
+    abstract public int sizeofNode(SerializationNode<?> node);
+
+    /**
+     * Serializes the node. All transition target positions must already have been defined. {@code payloadBits} must
+     * be four bits.
+     */
+    abstract public void serialize(DataOutputPlus out, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException;
+
+    // Implementations
+
+    final int bytesPerPointer;
+    static final int FRACTIONAL_BYTES = 0;
+
+    TrieNode(int ordinal, int bytesPerPointer)
+    {
+        this.ordinal = ordinal;
+        this.bytesPerPointer = bytesPerPointer;
+    }
+
+    final int ordinal;
+
+    static final TrieNode PAYLOAD_ONLY = new PayloadOnly();
+
+    static private class PayloadOnly extends TrieNode
+    {
+        // byte flags
+        // var payload
+        PayloadOnly()
+        {
+            super(0, FRACTIONAL_BYTES);
+        }
+
+        @Override
+        public int payloadPosition(ByteBuffer src, int position)
+        {
+            return position + 1;
+        }
+
+        @Override
+        public int search(ByteBuffer src, int position, int transitionByte)
+        {
+            return -1;
+        }
+
+        @Override
+        public long transitionDelta(ByteBuffer src, int position, int childIndex)
+        {
+            return 0;
+        }
+
+        @Override
+        public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+        {
+            return -1;
+        }
+
+        @Override
+        public long lastTransition(ByteBuffer src, int position, long positionLong)
+        {
+            return -1;
+        }
+
+        @Override
+        public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public int transitionByte(ByteBuffer src, int position, int childIndex)
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public int transitionRange(ByteBuffer src, int position)
+        {
+            return 0;
+        }
+
+        public int sizeofNode(SerializationNode<?> node)
+        {
+            return 1;
+        }
+
+        @Override
+        public void serialize(DataOutputPlus dest, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException
+        {
+            dest.writeByte((ordinal << 4) + (payloadBits & 0x0F));
+        }
+    }
+
+    static final TrieNode SINGLE_8 = new Single(2, 1);
+    static final TrieNode SINGLE_16 = new Single(4, 2);

Review Comment:
   I gave up and moved all the static instances to an embedded `Types` class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173518932


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format

Review Comment:
   Removed mentions and attempted support for legacy DSE versions --- the latter is more involved as the byte order conversion and stats metadata formats are not compatible.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173521378


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;

Review Comment:
   This change is now reverted as a deep recursion test started failing with `StackOverflowException`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178816028


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file

Review Comment:
   It _is_ wrong. I have no idea where this version came from; index is proportional to the number of keys, but the data file still has the full keys. Changed to the original calculation.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "da";
+
+        // versions aa-cz are not supported in OSS
+        // da - initial OSS version of the BIT format, Cassandra 5.0

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.
+            return -1;
+        }
+    }
+
+    protected void addIndexBlock() throws IOException
+    {
+        IndexInfo cIndexInfo = new IndexInfo(startPosition,

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.

Review Comment:
   Added a short explanation.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));

Review Comment:
   Added



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.
+            return -1;
+        }
+    }
+
+    protected void addIndexBlock() throws IOException
+    {
+        IndexInfo cIndexInfo = new IndexInfo(startPosition,
+                                             startOpenMarker);
+        rowTrie.add(firstClustering, lastClustering, cIndexInfo);
+        firstClustering = null;
+        ++rowIndexCount;
+    }
+
+    public int getRowIndexCount()

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)

Review Comment:
   The method's implementation is the same for both BIG and BTI, but it can't be expected to be the same for other formats, thus the implementation does not belong in `AbstractSSTableFormat`. Having each format call the shared implementation makes most sense.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;

Review Comment:
   `rowIndexBlockCount` and just `blockCount` in the `RowIndexEntry` hierarchy.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;

Review Comment:
   I went with `rowIndexBlockSize`.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();

Review Comment:
   Covered by `SimpleQueryTest` and others. Also added a new test in `TombstonesWithIndexedSSTableTest` to do some verification that deletions in index work correctly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185319582


##########
test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java:
##########
@@ -221,73 +219,4 @@ public void testOldReadsNew(String oldV, String newV) throws IOException
         }
     }
 
-    @Test
-    public void pendingRepairCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me").forEach(v -> assertFalse(format.getVersion(v).hasPendingRepair()));
-            Arrays.asList("na", "nb", "nc").forEach(v -> assertTrue(format.getVersion(v).hasPendingRepair()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void originatingHostCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "na").forEach(v -> assertFalse(format.getVersion(v).hasOriginatingHostId()));
-            Arrays.asList("me", "nb").forEach(v -> assertTrue(format.getVersion(v).hasOriginatingHostId()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void improvedMinMaxCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasImprovedMinMax()));
-            Arrays.asList("nc", "oa").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasImprovedMinMax()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void legacyMinMaxCompatiblity()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("oa").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasLegacyMinMax()));
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb", "nc").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasLegacyMinMax()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void partitionLevelDeletionPresenceMarkerCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasPartitionLevelDeletionsPresenceMarker()));
-            Arrays.asList("nc", "oa").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasPartitionLevelDeletionsPresenceMarker()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }

Review Comment:
   nit: Tests essentially moved to `AbstractTestVersionSupportedFeatures`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185545505


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/RowIndexTest.java:
##########
@@ -0,0 +1,517 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class RowIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(RowIndexTest.class);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static final Random RANDOM;
+
+    static
+    {
+        long seed = System.currentTimeMillis();
+        logger.info("seed = " + seed);
+        RANDOM = new Random(seed);
+
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    static final ClusteringComparator comparator = new ClusteringComparator(UUIDType.instance);
+    static final long END_MARKER = 1L << 40;
+    static final int COUNT = 8192;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    @Test
+    public void testSingletons() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        RowIndexReader summary = random.right;
+        List<ClusteringPrefix<?>> keys = random.left;
+        for (int i = 0; i < COUNT; i++)
+        {
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(i))).offset);
+        }
+        summary.close();
+    }
+
+    @Test
+    public void testSpans() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexQuads(COUNT);
+        RowIndexReader summary = random.right;
+        List<ClusteringPrefix<?>> keys = random.left;
+        int missCount = 0;
+        IndexInfo ii;
+        for (int i = 0; i < COUNT; i++)
+        {
+            // These need to all be within the span
+            assertEquals(i, (ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 1)))).offset);
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 2))).offset);
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 3))).offset);
+
+            // check other data
+            assertEquals(i + 2, ii.openDeletion.markedForDeleteAt());
+            assertEquals(i - 3, ii.openDeletion.localDeletionTime());
+
+            // before entry. hopefully here, but could end up in prev if matches prevMax too well
+            ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i)));
+            if (ii.offset != i)
+            {
+                ++missCount;
+                assertEquals(i - 1, ii.offset);
+            }
+        }
+        ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * COUNT)));
+        if (ii.offset != END_MARKER)
+        {
+            ++missCount;
+            assertEquals(COUNT - 1, ii.offset);
+        }
+        ii = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.BOTTOM));
+        assertEquals(0, ii.offset);
+
+        ii = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.TOP));
+        assertEquals(END_MARKER, ii.offset);
+
+        summary.close();
+        if (missCount > COUNT / 5)
+            logger.error("Unexpectedly high miss count: {}/{}", missCount, COUNT);
+    }
+
+    File file;
+    DataOutputStreamPlus dos;
+    RowIndexWriter writer;
+    FileHandle fh;
+    long root;
+
+    @After
+    public void cleanUp()
+    {
+        FileUtils.closeQuietly(dos);
+        FileUtils.closeQuietly(writer);
+        FileUtils.closeQuietly(fh);
+    }
+
+    public RowIndexTest() throws IOException
+    {
+        this(FileUtils.createTempFile("ColumnTrieReaderTest", ""));
+    }
+
+    RowIndexTest(File file) throws IOException
+    {
+        this(file, new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build()));
+    }
+
+    RowIndexTest(File file, DataOutputStreamPlus dos) throws IOException
+    {
+        this.file = file;
+        this.dos = dos;
+
+        // write some junk
+        dos.writeUTF("JUNK");
+        dos.writeUTF("JUNK");
+
+        writer = new RowIndexWriter(comparator, dos);
+    }
+
+    public void complete() throws IOException
+    {
+        root = writer.complete(END_MARKER);
+        dos.writeUTF("JUNK");
+        dos.writeUTF("JUNK");
+        dos.close();
+        dos = null;
+    }
+
+    public RowIndexReader completeAndRead() throws IOException
+    {
+        complete();
+
+        FileHandle.Builder builder = new FileHandle.Builder(file).mmapped(accessMode == Config.DiskAccessMode.mmap);
+        fh = builder.complete();
+        try (RandomAccessReader rdr = fh.createReader())
+        {
+            assertEquals("JUNK", rdr.readUTF());
+            assertEquals("JUNK", rdr.readUTF());
+        }
+        return new RowIndexReader(fh, root);
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        ClusteringPrefix<?> key = Clustering.EMPTY;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try (RowIndexReader summary = completeAndRead())
+        {
+            IndexInfo i = summary.min();
+            assertEquals(42, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.BOTTOM));
+            assertEquals(42, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.TOP));
+            assertEquals(END_MARKER, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(key));
+            assertEquals(42, i.offset);
+        }
+    }
+
+    @Test
+    public void testAddDuplicateEmptyThrow() throws Exception
+    {
+        ClusteringPrefix<?> key = Clustering.EMPTY;
+        Throwable t = null;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key, key, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testAddDuplicateThrow() throws Exception
+    {
+        ClusteringPrefix<?> key = generateRandomKey();
+        Throwable t = null;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key, key, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testAddOutOfOrderThrow() throws Exception
+    {
+        ClusteringPrefix<?> key1 = generateRandomKey();
+        ClusteringPrefix<?> key2 = generateRandomKey();
+        while (comparator.compare(key1, key2) <= 0) // make key2 smaller than 1
+            key2 = generateRandomKey();
+
+        Throwable t = null;
+        writer.add(key1, key1, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key2, key2, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        // This is not too relevant: due to the way we construct separators we can't be good enough on the left side.
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        List<ClusteringPrefix<?>> keys = random.left;
+
+        for (int i = 0; i < 500; ++i)
+        {
+            boolean exactLeft = RANDOM.nextBoolean();
+            boolean exactRight = RANDOM.nextBoolean();
+            ClusteringPrefix<?> left = exactLeft ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+            ClusteringPrefix<?> right = exactRight ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+            if (comparator.compare(right, left) < 0)
+            {
+                ClusteringPrefix<?> t = left;
+                left = right;
+                right = t;
+                boolean b = exactLeft;
+                exactLeft = exactRight;
+                exactRight = b;
+            }
+
+            try (RowIndexReverseIterator iter = new RowIndexReverseIterator(fh, root, comparator.asByteComparable(left), comparator.asByteComparable(right)))
+            {
+                IndexInfo indexInfo = iter.nextIndexInfo();
+                if (indexInfo == null)
+                {
+                    int idx = Collections.binarySearch(keys, right, comparator);
+                    if (idx < 0)
+                        idx = -2 - idx; // less than or equal
+                    if (idx <= 0)
+                        continue;
+                    assertTrue(comparator.asByteComparable(left) + " <= "
+                               + comparator.asByteComparable(keys.get(idx)) + " <= "
+                               + comparator.asByteComparable(right) + " but " + idx + " wasn't iterated.",
+                               comparator.compare(left, keys.get(idx - 1)) > 0);
+                    continue;
+                }
+
+                int idx = (int) indexInfo.offset;
+                if (indexInfo.offset == END_MARKER)
+                    idx = keys.size();
+                if (idx > 0)
+                    assertTrue(comparator.compare(right, keys.get(idx - 1)) > 0);
+                if (idx < keys.size() - 1)
+                    assertTrue(comparator.compare(right, keys.get(idx + 1)) < 0);
+                if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                    assertEquals(right, keys.get(idx));
+                while (true)
+                {
+                    --idx;
+                    IndexInfo ii = iter.nextIndexInfo();
+                    if (ii == null)
+                        break;
+                    assertEquals(idx, (int) ii.offset);
+                }
+                ++idx; // seek at last returned
+                if (idx < keys.size() - 1)
+                    assertTrue(comparator.compare(left, keys.get(idx + 1)) < 0);
+                // Because of the way we build the index (using non-prefix separator) we are usually going to miss the last item.
+                if (idx >= 2)
+                    assertTrue(comparator.compare(left, keys.get(idx - 2)) > 0);
+            }
+            catch (AssertionError e)
+            {
+                logger.error(e.getMessage(), e);
+                ClusteringPrefix<?> ll = left;
+                ClusteringPrefix<?> rr = right;
+                logger.info(keys.stream()
+                                .filter(x -> comparator.compare(ll, x) <= 0 && comparator.compare(x, rr) <= 0)
+                                .map(clustering -> comparator.asByteComparable(clustering))
+                                .map(bc -> bc.byteComparableAsString(VERSION))
+                                .collect(Collectors.joining(", ")));
+                logger.info("Left {}{} Right {}{}", comparator.asByteComparable(left), exactLeft ? "#" : "", comparator.asByteComparable(right), exactRight ? "#" : "");
+                try (RowIndexReverseIterator iter2 = new RowIndexReverseIterator(fh, root, comparator.asByteComparable(left), comparator.asByteComparable(right)))
+                {
+                    IndexInfo ii;
+                    while ((ii = iter2.nextIndexInfo()) != null)
+                    {
+                        logger.info(comparator.asByteComparable(keys.get((int) ii.offset)).toString());
+                    }
+                    logger.info("Left {}{} Right {}{}", comparator.asByteComparable(left), exactLeft ? "#" : "", comparator.asByteComparable(right), exactRight ? "#" : "");
+                }
+                throw e;
+            }
+        }
+    }
+
+    @Test
+    public void testReverseIteration() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        List<ClusteringPrefix<?>> keys = random.left;
+
+        for (int i = 0; i < 1000; ++i)
+        {
+            boolean exactRight = RANDOM.nextBoolean();
+            ClusteringPrefix<?> right = exactRight ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+
+            int idx = 0;
+            try (RowIndexReverseIterator iter = new RowIndexReverseIterator(fh, root, ByteComparable.EMPTY, comparator.asByteComparable(right)))
+            {
+                IndexInfo indexInfo = iter.nextIndexInfo();
+                if (indexInfo == null)
+                {
+                    idx = Collections.binarySearch(keys, right, comparator);
+                    if (idx < 0)
+                        idx = -2 - idx; // less than or equal
+                    assertTrue(comparator.asByteComparable(keys.get(idx)) + " <= "
+                               + comparator.asByteComparable(right) + " but " + idx + " wasn't iterated.",
+                               idx < 0);

Review Comment:
   Static analysis is telling me this is always `false`, but it seems like idx could still be `-1`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185289651


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException

Review Comment:
   nit: Seems like the duplication is intended, so maybe throw a `@SuppressWarnings("DuplicatedCode")` on there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185486855


##########
test/unit/org/apache/cassandra/io/tries/WalkerTest.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.TailOverridingRebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+
+@SuppressWarnings("unchecked")
+public class WalkerTest extends AbstractTrieTestBase
+{
+    @Test
+    public void testWithoutBounds() throws IOException
+    {
+        DataOutputBuffer buf = new DataOutputBufferPaged();
+        IncrementalTrieWriter<Integer> builder = makeTrie(buf);
+        long rootPos = builder.complete();
+
+        Rebufferer source = new ByteBufRebufferer(buf.asNewBuffer());
+
+        InternalIterator it = new InternalIterator(source, rootPos);
+
+        DataOutputBuffer dumpBuf = new DataOutputBuffer();
+        it.dumpTrie(new PrintStream(dumpBuf), (buf1, payloadPos, payloadFlags) -> String.format("%d/%d", payloadPos, payloadFlags));
+        logger.info("Trie dump: \n{}", new String(dumpBuf.getData()));
+        logger.info("Trie toString: {}", it);
+
+        it.goMax(rootPos);
+        assertEquals(7, it.payloadFlags());
+        assertEquals(TrieNode.Types.PAYLOAD_ONLY.ordinal, it.nodeTypeOrdinal());
+        assertEquals(1, it.nodeSize());
+        assertFalse(it.hasChildren());
+
+        it.goMin(rootPos);
+        assertEquals(1, it.payloadFlags());
+        assertEquals(TrieNode.Types.PAYLOAD_ONLY.ordinal, it.nodeTypeOrdinal());
+        assertEquals(1, it.nodeSize());
+        assertFalse(it.hasChildren());
+
+        assertEquals(-1, it.follow(source("151")));
+        assertEquals(2, it.payloadFlags());
+
+        assertEquals('3', it.follow(source("135")));
+
+        assertEquals('3', it.followWithGreater(source("135")));
+        it.goMin(it.greaterBranch);
+        assertEquals(2, it.payloadFlags());
+
+        assertEquals('3', it.followWithLesser(source("135")));
+        it.goMax(it.lesserBranch);
+        assertEquals(1, it.payloadFlags());
+
+        assertEquals(3, (Object) it.prefix(source("155"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertNull(it.prefix(source("516"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(5, (Object) it.prefix(source("5151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(1, (Object) it.prefix(source("1151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+
+        assertEquals(3, (Object) it.prefixAndNeighbours(source("155"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertNull(it.prefixAndNeighbours(source("516"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(5, (Object) it.prefixAndNeighbours(source("5151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(1, (Object) it.prefixAndNeighbours(source("1151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+
+        assertEquals(3, (Object) it.prefixAndNeighbours(source("1555"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        it.goMax(it.lesserBranch);
+        assertEquals(2, it.payloadFlags());
+        it.goMin(it.greaterBranch);
+        assertEquals(4, it.payloadFlags());
+    }
+
+    @Test
+    public void testWithBounds() throws IOException
+    {
+        DataOutputBuffer buf = new DataOutputBufferPaged();
+        IncrementalTrieWriter<Integer> builder = makeTrie(buf);
+        long rootPos = builder.complete();
+
+        Rebufferer source = new ByteBufRebufferer(buf.asNewBuffer());
+
+        InternalIterator it = new InternalIterator(source, rootPos, source("151"), source("515"), false);
+        long pos;
+        assertNotEquals(-1, pos = it.nextPayloadedNode());
+        assertEquals(3, TrieNode.at(buf.asNewBuffer(), (int) pos).payloadFlags(buf.asNewBuffer(), (int) pos));

Review Comment:
   This was a little confusing to me at first. "15" is clearly a prefix, but "151" is an exact match for an entry, right? The semantics of "admitting a prefix" in that case just looks like an "inclusive lower bound". In `testWithBoundsAndAdmitPrefix()`, we would also have a passing test with the left bound "15", which seems more like "admitting a prefix", but we start w/ "151" there as well.
   
   Am I thinking about this wrongly? :D



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185529566


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();

Review Comment:
   ```suggestion
       final IPartitioner partitioner = Util.testPartitioner();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182992726


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");

Review Comment:
   nit: Would the logging output in the scrubber here (and below) benefit from a standard format that includes the table name?
   
   ex. "[<keyspace>.<table>] Missing partition index. Continuing with scrub..."



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182992726


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");

Review Comment:
   nit: Would the logging output in the scrubber here (and below) benefit from a standard format that includes the table name?
   
   ex. "[keyspace.table] Missing partition index. Continuing with scrub..."



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183002922


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;
+                    }
+                }
+            }
+
+            String keyName = key == null ? "(unreadable key)" : keyString(key);
+            outputHandler.debug("partition %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSizeFromIndex));
+
+            try
+            {
+                if (key == null)
+                    throw new IOError(new IOException("Unable to read partition key from data file", keyReadError));
+
+                if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
+                {
+                    throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
+                                                                    //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
+                                                                    "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
+                }
+
+                if (indexIterator != null && dataSizeFromIndex > dataFile.length())
+                    throw new IOError(new IOException("Impossible partition size (greater than file length): " + dataSizeFromIndex));
+
+                if (indexIterator != null && dataStart != dataStartFromIndex)
+                    outputHandler.warn("Data file partition position %d differs from index file row position %d", dataStart, dataStartFromIndex);
+
+                if (tryAppend(prevKey, key, writer))
+                    prevKey = key;
+            }
+            catch (Throwable th)
+            {
+                throwIfFatal(th);
+                outputHandler.warn(th, "Error reading partition %s (stacktrace follows):", keyName);
+
+                if (currentIndexKey != null
+                    && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex))
+                {
+
+                    // position where the row should start in a data file (right after the partition key)
+                    long rowStartFromIndex = dataStartFromIndex + TypeSizes.SHORT_SIZE + currentIndexKey.remaining();
+                    outputHandler.output("Retrying from partition index; data is %s bytes starting at %s",
+                                         dataSizeFromIndex, rowStartFromIndex);
+                    key = sstable.decorateKey(currentIndexKey);
+                    try
+                    {
+                        if (!cfs.metadata.getLocal().isIndex())
+                            cfs.metadata.getLocal().partitionKeyType.validate(key.getKey());
+                        dataFile.seek(rowStartFromIndex);
+
+                        if (tryAppend(prevKey, key, writer))
+                            prevKey = key;
+                    }
+                    catch (Throwable th2)
+                    {
+                        throwIfFatal(th2);
+                        throwIfCannotContinue(key, th2);
+
+                        outputHandler.warn(th2, "Retry failed too. Skipping to next partition (retry's stacktrace follows)");
+                        badPartitions++;
+                        if (!seekToNextPartition())
+                            break;
+                    }
+                }
+                else
+                {
+                    throwIfCannotContinue(key, th);
+
+                    badPartitions++;
+                    if (indexIterator != null)
+                    {
+                        outputHandler.warn("Partition starting at position %d is unreadable; skipping to next", dataStart);
+                        if (!seekToNextPartition())
+                            break;
+                    }
+                    else
+                    {
+                        outputHandler.warn("Unrecoverable error while scrubbing %s." +
+                                           "Scrubbing cannot continue. The sstable will be marked for deletion. " +
+                                           "You can attempt manual recovery from the pre-scrub snapshot. " +
+                                           "You can also run nodetool repair to transfer the data from a healthy replica, if any.",
+                                           sstable);
+                        // There's no way to resync and continue. Give up.
+                        break;
+                    }
+                }
+            }
+        }
+    }
+
+
+    private boolean indexAvailable()
+    {
+        return indexIterator != null && !indexIterator.isExhausted();
+    }
+
+    private boolean seekToNextPartition()
+    {
+        while (indexAvailable())
+        {
+            long nextRowPositionFromIndex = indexIterator.dataPosition();
+
+            try
+            {
+                dataFile.seek(nextRowPositionFromIndex);
+                return true;
+            }
+            catch (Throwable th)
+            {
+                throwIfFatal(th);
+                outputHandler.warn(th, "Failed to seek to next row position %d", nextRowPositionFromIndex);
+                badPartitions++;
+            }
+
+            try
+            {
+                indexIterator.advance();
+            }
+            catch (Throwable th)
+            {
+                outputHandler.warn(th, "Failed to go to the next entry in index");
+                throw Throwables.cleaned(th);
+            }
+        }
+
+        return false;
+    }
+
+    @Override
+    protected void throwIfCannotContinue(DecoratedKey key, Throwable th)
+    {
+        if (isIndex)
+        {
+            outputHandler.warn("An error occurred while scrubbing the partition with key '%s' for an index table. " +
+                               "Scrubbing will abort for this table and the index will be rebuilt.", keyString(key));
+            throw new IOError(th);
+        }
+
+        super.throwIfCannotContinue(key, th);
+    }
+
+    public void close()

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183966836


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        @Override
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        @Override
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        @Override
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+
+            ClusteringBound<?> start = slice.start();
+            long currentPosition = file.getFilePointer();
+            foundLessThan = false;
+            // This is a copy of handlePreSliceData which also checks currentPosition < stopPosition.
+            // Not extracted to method as we need both marker and currentPosition.
+            if (filterStart)
+            {
+                while (currentPosition < stopPosition && deserializer.hasNext() && deserializer.compareNextTo(start) <= 0)
+                {
+                    if (deserializer.nextIsRow())
+                        deserializer.skipNext();
+                    else
+                        updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                    currentPosition = file.getFilePointer();
+                    foundLessThan = true;
+                }
+            }
+
+            // We've reached the beginning of our queried slice. If we have an open marker
+            // we should return that at the end of the slice to close the deletion.
+            if (openMarker != null)
+                blockOpenMarker = new RangeTombstoneBoundMarker(start, openMarker);
+
+
+            // Now deserialize everything until we reach our requested end (if we have one)
+            // See SSTableIterator.ForwardRead.computeNext() for why this is a strict inequality below: this is the same
+            // reasoning here.
+            while (currentPosition < stopPosition && deserializer.hasNext()
+                   && (!filterEnd || deserializer.compareNextTo(slice.end()) < 0))
+            {
+                rowOffsets.push(currentPosition);
+                if (deserializer.nextIsRow())
+                    deserializer.skipNext();
+                else
+                    updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                currentPosition = file.getFilePointer();
+            }
+
+            // If we have an open marker, we should output that first, unless end is not being filtered
+            // (i.e. it's either top (where a marker can't be open) or we placed that marker during previous block).
+            if (openMarker != null && filterEnd)
+            {
+                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                blockCloseMarker = new RangeTombstoneBoundMarker(slice.end(), openMarker);
+                openMarker = null;
+            }
+        }
+    }
+
+    private class ReverseIndexedReader extends ReverseReader
+    {
+        private RowIndexReverseIterator indexReader;
+        private final TrieIndexEntry indexEntry;
+        private final long basePosition;

Review Comment:
   nit: Not a big deal, but I wonder if naming this something like `partitionStart` or `partitionStartPosition` would make it more readable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180582480


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ * <p>
+ * The files created by this builder are read by {@link PartitionIndex}.
+ */
+class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings({ "resource", "RedundantSuppression" })
+            PartitionIndex pi = new PartitionIndexEarly(fh, partialIndexTail.root(), partialIndexTail.count(), firstKey, partialIndexLastKey, partialIndexTail.cutoff(), partialIndexTail.tail());
+            partialIndexConsumer.accept(pi);
+            partialIndexConsumer = null;
+        }
+        finally
+        {
+            fhBuilder.withLengthOverride(-1);
+        }
+
+    }
+
+    /**
+    * @param decoratedKey the key for this record
+    * @param position the position to write with the record:
+    *    - positive if position points to an index entry in the index file
+    *    - negative if ~position points directly to the key in the data file
+    */
+    public PartitionIndexBuilder addEntry(DecoratedKey decoratedKey, long position) throws IOException

Review Comment:
   nit: return value is never used?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180602259


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,428 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+/**
+ * Writes SSTables in BTI format (see {@link BtiFormat}), which can be read by {@link BtiTableReader}.
+ */
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexBlockCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+    @Override
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    @Override
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page.
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @Override
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    @Override
+    protected TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy(() -> FBUtilities.immutableListWithFilteredNulls(iwriter, dataWriter));
+    }
+
+    private class TransactionalProxy extends SortedTableWriter<BtiFormatPartitionWriter>.TransactionalProxy
+    {
+        public TransactionalProxy(Supplier<ImmutableList<Transactional>> transactionals)
+        {
+            super(transactionals);
+        }
+
+        @Override
+        protected Throwable doPostCleanup(Throwable accumulate)
+        {
+            accumulate = Throwables.close(accumulate, partitionWriter);
+            accumulate = super.doPostCleanup(accumulate);
+            return accumulate;
+        }
+    }
+
+    /**
+     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
+     */
+    static class IndexWriter extends SortedTableWriter.AbstractIndexWriter
+    {
+        final SequentialWriter rowIndexWriter;
+        private final FileHandle.Builder rowIndexFHBuilder;
+        private final SequentialWriter partitionIndexWriter;
+        private final FileHandle.Builder partitionIndexFHBuilder;
+        private final PartitionIndexBuilder partitionIndex;
+        boolean partitionIndexCompleted = false;
+        private DataPosition riMark;
+        private DataPosition piMark;
+
+        IndexWriter(Builder b)
+        {
+            super(b);
+            rowIndexWriter = new SequentialWriter(descriptor.fileFor(Components.ROW_INDEX), b.getIOOptions().writerOptions);
+            rowIndexFHBuilder = IndexComponent.fileBuilder(Components.ROW_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndexWriter = new SequentialWriter(descriptor.fileFor(Components.PARTITION_INDEX), b.getIOOptions().writerOptions);
+            partitionIndexFHBuilder = IndexComponent.fileBuilder(Components.PARTITION_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndex = new PartitionIndexBuilder(partitionIndexWriter, partitionIndexFHBuilder);
+            // register listeners to be alerted when the data files are flushed
+            partitionIndexWriter.setPostFlushListener(() -> partitionIndex.markPartitionIndexSynced(partitionIndexWriter.getLastFlushOffset()));
+            rowIndexWriter.setPostFlushListener(() -> partitionIndex.markRowIndexSynced(rowIndexWriter.getLastFlushOffset()));
+            @SuppressWarnings({"resource", "RedundantSuppression"})
+            SequentialWriter dataWriter = b.getDataWriter();
+            dataWriter.setPostFlushListener(() -> partitionIndex.markDataSynced(dataWriter.getLastFlushOffset()));
+        }
+
+        public long append(DecoratedKey key, AbstractRowIndexEntry indexEntry) throws IOException
+        {
+            bf.add(key);
+            long position;
+            if (indexEntry.isIndexed())
+            {
+                long indexStart = rowIndexWriter.position();
+                try
+                {
+                    ByteBufferUtil.writeWithShortLength(key.getKey(), rowIndexWriter);
+                    ((TrieIndexEntry) indexEntry).serialize(rowIndexWriter, rowIndexWriter.position());
+                }
+                catch (IOException e)
+                {
+                    throw new FSWriteError(e, rowIndexWriter.getFile());
+                }
+
+                if (logger.isTraceEnabled())
+                    logger.trace("wrote index entry: {} at {}", indexEntry, indexStart);
+                position = indexStart;
+            }
+            else
+            {
+                // Write data position directly in trie.
+                position = ~indexEntry.position;
+            }
+            partitionIndex.addEntry(key, position);
+            return position;
+        }
+
+        public boolean buildPartial(long dataPosition, Consumer<PartitionIndex> callWhenReady)

Review Comment:
   nit: unused return value? (not a big deal, just want to make sure it's not an oversight)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182097183


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()
+    {
+        return new BtiScanningIterator();
+    }
+
+    protected class BtiScanningIterator extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>.BaseKeyScanningIterator implements Closeable
+    {
+        private PartitionIterator iterator;
+
+        protected boolean prepareToIterateRow() throws IOException
+        {
+            while (true)
+            {
+                if (startScan != -1)
+                    bytesScanned += getCurrentPosition() - startScan;
+
+                if (iterator != null)
+                {
+                    currentEntry = iterator.entry();
+                    currentKey = iterator.decoratedKey();
+                    if (currentEntry != null)
+                    {
+                        iterator.advance();
+                        return true;
+                    }
+                    iterator.close();
+                    iterator = null;
+                }
+
+                // try next range
+                if (!rangeIterator.hasNext())
+                    return false;
+                iterator = sstable.coveredKeysIterator(rangeIterator.next());
+            }
+        }
+
+        protected UnfilteredRowIterator getRowIterator(TrieIndexEntry indexEntry, DecoratedKey key)

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182093964


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185507453


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));

Review Comment:
   ```suggestion
                           assertSame(left, keys.get(idx));
   ```
   nit: produces a nicer error I guess



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162044295


##########
conf/cassandra.yaml:
##########
@@ -1924,8 +1924,14 @@ drop_compact_storage_enabled: false
 # which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
 # which will be used to recognize the format type - in particular that name will be used in sstable file names and in
 # stream headers so the name has to be the same for the same format across all the nodes in the cluster.
+# The first entry in this list is the format that will be used for newly-created SSTables. The other formats given
+# will be used to read any SSTables present in the data directories or streamed.
 sstable_formats:
   - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
     parameters:
       id: 0
       name: big
+  - class_name: org.apache.cassandra.io.sstable.format.bti.BtiFormat
+    parameters:
+      id: 1
+      name: bti

Review Comment:
   Agree with what @maedhroz said, and to extend on that.
   
   1) We figure out which format to use by using `org.apache.cassandra.io.sstable.format.SSTableFormat.Type#current`, which uses the first element of the list, which is not determinist as it is based off `Map` ordering (see `org.apache.cassandra.io.sstable.format.SSTableFormat.Type#readFactories`)
   1.1) we use a system property to override, but we should be using a config in cassandra.yml.  cassandra.yml is accessible via system properties if that is what a user wants, but also plugs in well with the rest of the system.
   1.2) if we do go the config route, we can make this a hot-prop using the existing config system
   2) streaming now requires globally consistent configs... this isn't realistic without transactional cluster metadata, so cases such as `org.apache.cassandra.db.streaming.CassandraStreamHeader.CassandraStreamHeaderSerializer#serialize` can have one instance writing "abc" and the other side failing as it doesn't know about "abc" (because it defined it as "big").
   
   I strongly feel we should redo this config layer and remove id/name from operator control as this isn't safe.  
   
   Now, if you want to add a custom format (I believe one of the goals of the pluggable work) then we need a "safe" way to discover, and I feel the only real safe way is discovery is via class existence (such as `ServiceLoader` from java).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167198123


##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;

Review Comment:
   nit: `limit` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167197664


##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;

Review Comment:
   nit: `node` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167287819


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/TrieIndexFormatUtil.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FilterFactory;
+
+import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
+
+public class TrieIndexFormatUtil

Review Comment:
   class unused?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1168097632


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE

Review Comment:
   we should remove informations about "DSE" in apache cassandra 



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (IOException e)
+        {
+            outputHandler.warn("Index is unreadable.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th, "Failed to advance to the next index position. Index is corrupted. " +

Review Comment:
   the code format here seems not right ?



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE

Review Comment:
   remove this comment too



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);
+
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new UnsupportedOperationException("Unsupported op: " + operator);

Review Comment:
   I think  we should enrich the information  of the exception like Unsuppted op: for Bti sstable or Trie-indexed SSTable



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public boolean hasIsTransient()
+        {
+            return version.compareTo("ca") >= 0;
+        }
+
+        @Override
+        public boolean hasMetadataChecksum()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasOldBfFormat()
+        {
+            return hasOldBfFormat;
+        }
+
+        @Override
+        public boolean hasAccurateMinMax()
+        {
+            return hasAccurateMinMax;
+        }
+
+        public boolean hasLegacyMinMax()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean hasOriginatingHostId()
+        {
+            return hasOriginatingHostId;
+        }
+
+        @Override
+        public boolean hasImprovedMinMax() {
+            return hasImprovedMinMax;
+        }
+
+        @Override
+        public boolean hasPartitionLevelDeletionsPresenceMarker()
+        {
+            return hasPartitionLevelDeletionsPresenceMarker;
+        }
+
+        @Override
+        public boolean hasKeyRange()
+        {
+            return hasKeyRange;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);
+        }
+
+        // this field is not present in DSE

Review Comment:
   remove this comment



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public boolean hasIsTransient()
+        {
+            return version.compareTo("ca") >= 0;
+        }
+
+        @Override
+        public boolean hasMetadataChecksum()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasOldBfFormat()
+        {
+            return hasOldBfFormat;
+        }
+
+        @Override
+        public boolean hasAccurateMinMax()
+        {
+            return hasAccurateMinMax;
+        }
+
+        public boolean hasLegacyMinMax()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean hasOriginatingHostId()
+        {
+            return hasOriginatingHostId;
+        }
+
+        @Override
+        public boolean hasImprovedMinMax() {
+            return hasImprovedMinMax;
+        }
+
+        @Override
+        public boolean hasPartitionLevelDeletionsPresenceMarker()
+        {
+            return hasPartitionLevelDeletionsPresenceMarker;
+        }
+
+        @Override
+        public boolean hasKeyRange()
+        {
+            return hasKeyRange;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);

Review Comment:
   I think this support version should have a ut case when the current_version and earliest_supported_version change to apache cassandra support version format.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171699955


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,1010 @@
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+ 
+     http://www.apache.org/licenses/LICENSE-2.0
+ 
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+# Big Trie-Indexed (BTI) SSTable format
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.com/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size`
+`cassandra.yaml` parameter).
+
+Our row index implementation thus creates a map from clustering keys or
+prefixes to the data position at the start of the index block which is
+the earliest that could contain a key equal or greater than the given
+one. Additionally, if there is an active deletion at the beginning of
+the block, the index must specify it so that it can be taken into
+account when merging data from multiple sstables.
+
+Each index block will contain at least one key, but generally it will
+have different first and last keys. We don't store these keys, but
+instead we index the positions between blocks by storing a "separator",
+some key that is greater than the last key of the previous block and
+smaller than or equal to the first key of the next[^4]. Then, when we
+look up a given clustering, we follow its bytes as long as we can in the
+trie and we can be certain that all blocks before the closest
+less-than-or-equal entry in the trie cannot contain any data that is
+greater than or equal to the given key.
+
+[^4]: Another way to interpret this is that we index the start of each
+block only, but for efficiency we don't use the first key of the
+block as its beginning, but instead something closer to the last key
+of the previous block (but still greater than it).
+
+It may happen that the identified block actually doesn't contain any
+matching data (e.g. because the looked-up key ends up between the last
+key in the block and the separator), but this only affects efficiency as
+the iteration mechanism does not expect the data position returned by
+the index to be guaranteed to start with elements that fit the criteria;
+it would only have to walk a whole block forward to find the matching
+key.
+
+It is important to keep the number of these false positives low, and at
+the same time we aim for the smallest possible size of the index for a
+given granularity. The choice of separator affects this balance[^5]; the
+option we use, as a good tradeoff in the vein of the unique prefix
+approach used in the partition index, is to use the shortest prefix of
+the next block's beginning key that separates it from the previous
+block's end key, adjusted so that the last byte of it is 1 greater than
+that end key.
+
+[^5]: For example, the best separator for false positives is the next
+possible byte sequence after the previous block's final key, which
+is obtained by adding a 00 byte to its end. This, however, means all
+the bytes of the byte-ordered representation of this key must be
+present in the index, which inflates the index's size and lookup
+complexity.
+
+For example, if block 2 covers "something" to "somewhere" and block 3
+&mdash; "sorry" to "tease", then the sequence "son" is used as the separator
+between blocks 2 and 3. This leaves things like "sommelier" in the area
+that triggers false positives, but stores and has to walk just three
+bytes to find the starting point for iteration.
+
+### Efficiency
+
+Finding the candidate block in the trie involves walking the byte
+ordered representation of the clustering key in the trie and finding the
+closest less-than-or-equal value. The number of steps is proportional to
+the length of the separators &mdash; the lower their number the shorter that
+sequence is, though we can't expect _O_(log _n_) complexity since there may
+be many items sharing the same long prefixes (e.g. if there are long
+strings in the components of the clustering keys before the last). Even
+so, such repeating prefixes are addressed very well by the page-packing
+and `SINGLE_NOPAYLOAD_4` node type, resulting in very efficient walks.
+
+After this step we also perform a linear walk within the data file to
+find the actual start of the matching data. This is usually costlier and
+may involve object allocation and deserialization.
+
+The tradeoff between the size of the index and the time it takes to find
+the relevant rows is controlled by the index granularity. The lower it
+is, the more efficient lookup (especially exact match lookup) becomes at
+the expense of bigger index size. The 16kb default is chosen pretty
+conservatively[^6]; if users don't mind bigger indices something like 4,
+2 or 1kb granularity should be quite a bit more efficient. It is also
+possible to index every row by choosing a granularity of 0kb; at these
+settings in-cache trie-indexed sstables tend to outperform
+`ConcurrentSkipListMap` memtables for reads.
+
+[^6]: This was chosen with the aim to match the size of the trie index
+compared to the earlier version of the row index at its default
+granularity of 64kb.
+
+### Reverse lookup
+
+To perform a reverse lookup, we can use the same mechanism as above
+(with greater-than-or-equal) to find the initial block for the
+iteration. However, in the forward direction we could simply walk the
+data file to find the next rows, but this isn't possible going
+backwards.
+
+To solve this problem the index helps the iteration machinery by
+providing an iterator of index blocks in reverse order. For each index
+block the iteration walks it forward and creates a stack of all its row
+positions, then starts issuing rows by popping and examining rows from
+that stack. When the stack is exhausted it requests the previous block
+from the index and applies the same procedure there.
+
+# Code structure
+
+The implementation is mostly in two packages, `o.a.c.io.tries` contains
+the generic code to construct and read on-disk tries, and 
+`o.a.c.io.sstable.format.bti`, which implements the specifics of the
+format and the two indexes.
+
+## Building tries
+
+Tries are built from sorted keys using an [`IncrementalTrieWriter`](../../../tries/IncrementalTrieWriter.java). 
+The code contains three implementations with increasing complexity:
+- [`IncrementalTrieWriterSimple`](../../../tries/IncrementalTrieWriterSimple.java)
+  implements simple incremental construction of tries from sorted input,
+- [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalTrieWriterPageAware.java)
+  adds packing of nodes to disk pages,
+- [`IncrementalDeepTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java)
+  adds the ability to transition to on-heap recursion for all stages of the construction
+  process to be able to handle very large keys.
+
+Only the latter is used, but we provide (and test) the other two as a form of
+documentation.
+
+The builders take a `TrieSerializer` as parameter, which determines how the nodes
+are written. The indexes implement this using `TrieNode`, writing any payload they
+need immediately after the node serialization.
+
+## Reading tries
+
+The BTI format tries are used directly in their on-disk format. To achieve this,
+all node types are implemented as static objects in `TrieNode`. Reading nodes in
+a file is encapsulated in [`Walker`](../../../tries/Walker.java), 
+which provides a method to `go` to a specific node and use it, i.e. 
+get any associated data, search in the children list and
+follow transitions to children. It also provides functionality to find the
+mapping for a given key, floors and ceilings as well as some combinations.
+Iterating the payloads between two key bounds is implemented by 
+[`ValueIterator`](../../../tries/ValueIterator.java),
+and [`ReverseValueIterator`](../../../tries/ReverseValueIterator.java).
+
+Special care is given to prefixes to make sure the semantics of searches matches
+what the format needs.
+
+## SSTable format implementation
+
+The two indexes are implemented, respectively, by [`PartitionIndex`](PartitionIndex.java)
+/[`PartitionIndexBuilder`](PartitionIndexBuilder.java)
+and [`RowIndexReader`](RowIndexReader.java)/[`RowIndexWriter`](RowIndexWriter.java). 
+The format implementation extends the filtered
+base class and follows the structure of the BIG implementation, where
+all references to the primary index are replaced with calls to these two 
+classes.
+
+# Index file format in BTI
+
+## Trie nodes
+Implemented in [`TrieNode.java`](../../../tries/TrieNode.java)
+
+Nodes start with four bits of node type, followed by 4 payload bits
+(_pb_), which are 0 if the node has no associated payload; otherwise the
+node type gives an option to compute the starting position for the
+payload (_ppos_) from the starting position of the node (_npos_).
+The layout of the node depends on its type.
+
+`PAYLOAD_ONLY` nodes:
+
+-   4 type bits, 0
+
+-   4 payload bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 1
+
+`SINGLE_NOPAYLOAD_4` and `SINGLE_NOPAYLOAD_12` nodes:
+
+-   4 type bits
+
+-   4 pointer bits
+
+-   8 pointer bits (for `SINGLE_NOPAYLOAD_12`)
+
+-   8 bits transition byte
+
+-   _pb_ is assumed 0
+
+`SINGLE_8/16`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bits transition byte
+
+-   8/16 pointer bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 3/4
+
+`SPARSE_8/12/16/24/40`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bit child count
+
+-   8 bits per child, the transition bytes
+
+-   8/12/16/24/40 bits per child, the pointers
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 2 + (2/2.5/3/4/6)*(_child
+    count_) (rounded up)
+
+`DENSE_12/16/24/32/40/LONG`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bit start byte value
+
+-   8 bit _length_-1
+
+-   _length_ * 12/16/24/32/40/64 bits per child, the pointers
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 3 + (1.5/2/3/4/5/8)*(_length_)
+    (rounded up)
+
+This is the space taken by each node type (_CS_ stands for child span,
+i.e. largest - smallest + 1, _CC_ is child count):
+
+|Type                 | Size in bytes excl. payload |Size for 1 child|Size for 9 dense children (01-08, 10)|Size for 10 sparse children (01 + i*10)|Why the type is needed          |
+|:--------------------|:----------------------------|---------------:|-------------------:|-------------------:|:-------------------------------|
+|`PAYLOAD_ONLY`       | 1                           | -              | -                  | -                  |  Leaves dominate the trie      |
+|`SINGLE_NOPAYLOAD_4` | 2                           |2               | -                  | -                  |  Single-transition chains      |
+|`SINGLE_8`           | 3                           |3               | -                  | -                  |  Payload within chain          |
+|`SPARSE_8`           | 2 + _CC_ * 2                |4               | 20                 | 22                 |  Most common type after leaves |
+|`SINGLE_NOPAYLOAD_12`| 3                           |3               | -                  | -                  |  12 bits cover all in-page transitions    | 
+|`SPARSE_12`          | 2 + _CC_ * 2.5              |5               | 25                 | 27                 |  Size of sparse is proportional to number of children |
+|`DENSE_12`           | 3 + _CS_ * 1.5              |5               | 18                 | 140                |  Lookup in dense is faster, size smaller if few holes    | 
+|`SINGLE_16`          | 4                           |4               | -                  | -                  |                                |    
+|`SPARSE_16`          | 2 + _CC_ * 3                |5               | 29                 | 32                 |                                |     
+|`DENSE_16`           | 3 + _CS_ * 2                |5               | 23                 | 185                |                                |     
+|`SPARSE_24`          | 2 + _CC_ * 4                |6               | 38                 | 42                 |                                |     
+|`DENSE_24`           | 3 + _CS_ * 3                |6               | 33                 | 276                |                                |     
+|`DENSE_32`           | 3 + _CS_ * 4                |7               | 43                 | 367                |  Nodes with big subtrees are usually dense   | 
+|`SPARSE_40`          | 2 + _CC_ * 6                |8               | 56                 | 62                 |                                |     
+|`DENSE_40`           | 3 + _CS_ * 5                |8               | 53                 | 458                |                                |     
+|`DENSE_LONG`         | 3 + _CS_ * 8                |11              | 83                 | 731                |  Catch-all                     |
+
+All pointers are stored as distances, and since all tries are written
+from the bottom up (and hence a child is always before the parent in the
+file), the distance is subtracted from the position of the current node
+to obtain the position of the child node.
+
+Note: All nodes are placed in such a way that they do not cross a page
+boundary. I.e. if a reader (e.g. [`Walker`](../../../tries/Walker.java)) 
+is positioned at a node, it is
+guaranteed that all reads of the node's data can complete without
+requiring a different page to be fetched from disk.
+
+## Partition index
+Implemented in [`PartitionIndex.java`](PartitionIndex.java)
+
+Layout:
+```
+[nodes page, 4096 bytes]
+...
+[nodes page, 4096 bytes]
+[nodes page including root node, ≤4096 bytes]
+[smallest key, with short length]
+[largest key, with short length]
+[smallest key pos, long]
+[key count, long]
+[root pos, long]
+```
+
+The SSTable's partition index is stored in the -Partitions.db file. The
+file itself is written from the bottom up, and its "header" is at the
+end of the file.
+
+More precisely, the last 3 longs in the file contain:
+
+-   A file position where the smallest and greatest key are written.
+
+-   The exact number of keys in the file.
+
+-   A file position for the root node of the index.
+
+These three longs are preceded by the serialization of the first and
+last key, and before that are the trie contents.
+
+To find a match for the key, start at the root position, decode the node
+(see the "Trie nodes" section above) and follow the transitions
+according to the bytes of the byte-ordered representation of the key
+while the node has children and there are bytes left in the key.
+
+If a leaf node is reached, that node contains the following payload:
+
+-   If _pb_ < 8, let
+
+    -   _idxpos_ be the sign-extended integer value of length _pb_ at
+        _ppos_
+
+-   If _pb_ &ge; 8 (always the case in DSE 6 files), let

Review Comment:
   nit: Remove mention of DSE



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178402976


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));

Review Comment:
   nit: Perhaps a quick message would be helpful indicating the tables don't match?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178359354


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "da";
+
+        // versions aa-cz are not supported in OSS
+        // da - initial OSS version of the BIT format, Cassandra 5.0

Review Comment:
   ```suggestion
           // da (5.0): initial version of the BTI format
   ```
   ...just for parity w/ BigFormat's docs ;)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178379175


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)

Review Comment:
   Should we just pull this method up to `AbstractSSTableFormat`? (Same for `BigFormat` I think...)
   
   EDIT: ...or just have things call `SortedTableScrubber.deleteOrphanedComponents()` directly, since there's nothing format-specific going on?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167090621


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;

Review Comment:
   I suppose we're aliasing just for readability here, although if we keep the `if` one line 118, we should probably use `filterNeeded` there as well?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171198000


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;

Review Comment:
   Refactored



##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException

Review Comment:
   Removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1172477692


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,1010 @@
+<!---
+ Licensed to the Apache Software Foundation (ASF) under one
+ or more contributor license agreements.  See the NOTICE file
+ distributed with this work for additional information
+ regarding copyright ownership.  The ASF licenses this file
+ to you under the Apache License, Version 2.0 (the
+ "License"); you may not use this file except in compliance
+ with the License.  You may obtain a copy of the License at
+ 
+     http://www.apache.org/licenses/LICENSE-2.0
+ 
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+
+# Big Trie-Indexed (BTI) SSTable format
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.com/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size`
+`cassandra.yaml` parameter).
+
+Our row index implementation thus creates a map from clustering keys or
+prefixes to the data position at the start of the index block which is
+the earliest that could contain a key equal or greater than the given
+one. Additionally, if there is an active deletion at the beginning of
+the block, the index must specify it so that it can be taken into
+account when merging data from multiple sstables.
+
+Each index block will contain at least one key, but generally it will
+have different first and last keys. We don't store these keys, but
+instead we index the positions between blocks by storing a "separator",
+some key that is greater than the last key of the previous block and
+smaller than or equal to the first key of the next[^4]. Then, when we
+look up a given clustering, we follow its bytes as long as we can in the
+trie and we can be certain that all blocks before the closest
+less-than-or-equal entry in the trie cannot contain any data that is
+greater than or equal to the given key.
+
+[^4]: Another way to interpret this is that we index the start of each
+block only, but for efficiency we don't use the first key of the
+block as its beginning, but instead something closer to the last key
+of the previous block (but still greater than it).
+
+It may happen that the identified block actually doesn't contain any
+matching data (e.g. because the looked-up key ends up between the last
+key in the block and the separator), but this only affects efficiency as
+the iteration mechanism does not expect the data position returned by
+the index to be guaranteed to start with elements that fit the criteria;
+it would only have to walk a whole block forward to find the matching
+key.
+
+It is important to keep the number of these false positives low, and at
+the same time we aim for the smallest possible size of the index for a
+given granularity. The choice of separator affects this balance[^5]; the
+option we use, as a good tradeoff in the vein of the unique prefix
+approach used in the partition index, is to use the shortest prefix of
+the next block's beginning key that separates it from the previous
+block's end key, adjusted so that the last byte of it is 1 greater than
+that end key.
+
+[^5]: For example, the best separator for false positives is the next
+possible byte sequence after the previous block's final key, which
+is obtained by adding a 00 byte to its end. This, however, means all
+the bytes of the byte-ordered representation of this key must be
+present in the index, which inflates the index's size and lookup
+complexity.
+
+For example, if block 2 covers "something" to "somewhere" and block 3
+&mdash; "sorry" to "tease", then the sequence "son" is used as the separator
+between blocks 2 and 3. This leaves things like "sommelier" in the area
+that triggers false positives, but stores and has to walk just three
+bytes to find the starting point for iteration.
+
+### Efficiency
+
+Finding the candidate block in the trie involves walking the byte
+ordered representation of the clustering key in the trie and finding the
+closest less-than-or-equal value. The number of steps is proportional to
+the length of the separators &mdash; the lower their number the shorter that
+sequence is, though we can't expect _O_(log _n_) complexity since there may
+be many items sharing the same long prefixes (e.g. if there are long
+strings in the components of the clustering keys before the last). Even
+so, such repeating prefixes are addressed very well by the page-packing
+and `SINGLE_NOPAYLOAD_4` node type, resulting in very efficient walks.
+
+After this step we also perform a linear walk within the data file to
+find the actual start of the matching data. This is usually costlier and
+may involve object allocation and deserialization.
+
+The tradeoff between the size of the index and the time it takes to find
+the relevant rows is controlled by the index granularity. The lower it
+is, the more efficient lookup (especially exact match lookup) becomes at
+the expense of bigger index size. The 16kb default is chosen pretty
+conservatively[^6]; if users don't mind bigger indices something like 4,
+2 or 1kb granularity should be quite a bit more efficient. It is also
+possible to index every row by choosing a granularity of 0kb; at these
+settings in-cache trie-indexed sstables tend to outperform
+`ConcurrentSkipListMap` memtables for reads.
+
+[^6]: This was chosen with the aim to match the size of the trie index
+compared to the earlier version of the row index at its default
+granularity of 64kb.
+
+### Reverse lookup
+
+To perform a reverse lookup, we can use the same mechanism as above
+(with greater-than-or-equal) to find the initial block for the
+iteration. However, in the forward direction we could simply walk the
+data file to find the next rows, but this isn't possible going
+backwards.
+
+To solve this problem the index helps the iteration machinery by
+providing an iterator of index blocks in reverse order. For each index
+block the iteration walks it forward and creates a stack of all its row
+positions, then starts issuing rows by popping and examining rows from
+that stack. When the stack is exhausted it requests the previous block
+from the index and applies the same procedure there.
+
+# Code structure
+
+The implementation is mostly in two packages, `o.a.c.io.tries` contains
+the generic code to construct and read on-disk tries, and 
+`o.a.c.io.sstable.format.bti`, which implements the specifics of the
+format and the two indexes.
+
+## Building tries
+
+Tries are built from sorted keys using an [`IncrementalTrieWriter`](../../../tries/IncrementalTrieWriter.java). 
+The code contains three implementations with increasing complexity:
+- [`IncrementalTrieWriterSimple`](../../../tries/IncrementalTrieWriterSimple.java)
+  implements simple incremental construction of tries from sorted input,
+- [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalTrieWriterPageAware.java)
+  adds packing of nodes to disk pages,
+- [`IncrementalDeepTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java)
+  adds the ability to transition to on-heap recursion for all stages of the construction
+  process to be able to handle very large keys.
+
+Only the latter is used, but we provide (and test) the other two as a form of
+documentation.
+
+The builders take a `TrieSerializer` as parameter, which determines how the nodes
+are written. The indexes implement this using `TrieNode`, writing any payload they
+need immediately after the node serialization.
+
+## Reading tries
+
+The BTI format tries are used directly in their on-disk format. To achieve this,
+all node types are implemented as static objects in `TrieNode`. Reading nodes in
+a file is encapsulated in [`Walker`](../../../tries/Walker.java), 
+which provides a method to `go` to a specific node and use it, i.e. 
+get any associated data, search in the children list and
+follow transitions to children. It also provides functionality to find the
+mapping for a given key, floors and ceilings as well as some combinations.
+Iterating the payloads between two key bounds is implemented by 
+[`ValueIterator`](../../../tries/ValueIterator.java),
+and [`ReverseValueIterator`](../../../tries/ReverseValueIterator.java).
+
+Special care is given to prefixes to make sure the semantics of searches matches
+what the format needs.
+
+## SSTable format implementation
+
+The two indexes are implemented, respectively, by [`PartitionIndex`](PartitionIndex.java)
+/[`PartitionIndexBuilder`](PartitionIndexBuilder.java)
+and [`RowIndexReader`](RowIndexReader.java)/[`RowIndexWriter`](RowIndexWriter.java). 
+The format implementation extends the filtered
+base class and follows the structure of the BIG implementation, where
+all references to the primary index are replaced with calls to these two 
+classes.
+
+# Index file format in BTI
+
+## Trie nodes
+Implemented in [`TrieNode.java`](../../../tries/TrieNode.java)
+
+Nodes start with four bits of node type, followed by 4 payload bits
+(_pb_), which are 0 if the node has no associated payload; otherwise the
+node type gives an option to compute the starting position for the
+payload (_ppos_) from the starting position of the node (_npos_).
+The layout of the node depends on its type.
+
+`PAYLOAD_ONLY` nodes:
+
+-   4 type bits, 0
+
+-   4 payload bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 1
+
+`SINGLE_NOPAYLOAD_4` and `SINGLE_NOPAYLOAD_12` nodes:
+
+-   4 type bits
+
+-   4 pointer bits
+
+-   8 pointer bits (for `SINGLE_NOPAYLOAD_12`)
+
+-   8 bits transition byte
+
+-   _pb_ is assumed 0
+
+`SINGLE_8/16`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bits transition byte
+
+-   8/16 pointer bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 3/4
+
+`SPARSE_8/12/16/24/40`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bit child count
+
+-   8 bits per child, the transition bytes
+
+-   8/12/16/24/40 bits per child, the pointers
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 2 + (2/2.5/3/4/6)*(_child
+    count_) (rounded up)
+
+`DENSE_12/16/24/32/40/LONG`:
+
+-   4 type bits
+
+-   4 payload bits
+
+-   8 bit start byte value
+
+-   8 bit _length_-1
+
+-   _length_ * 12/16/24/32/40/64 bits per child, the pointers
+
+-   payload if _pb_ &ne; 0, _ppos_ is _npos_ + 3 + (1.5/2/3/4/5/8)*(_length_)
+    (rounded up)
+
+This is the space taken by each node type (_CS_ stands for child span,
+i.e. largest - smallest + 1, _CC_ is child count):
+
+|Type                 | Size in bytes excl. payload |Size for 1 child|Size for 9 dense children (01-08, 10)|Size for 10 sparse children (01 + i*10)|Why the type is needed          |
+|:--------------------|:----------------------------|---------------:|-------------------:|-------------------:|:-------------------------------|
+|`PAYLOAD_ONLY`       | 1                           | -              | -                  | -                  |  Leaves dominate the trie      |
+|`SINGLE_NOPAYLOAD_4` | 2                           |2               | -                  | -                  |  Single-transition chains      |
+|`SINGLE_8`           | 3                           |3               | -                  | -                  |  Payload within chain          |
+|`SPARSE_8`           | 2 + _CC_ * 2                |4               | 20                 | 22                 |  Most common type after leaves |
+|`SINGLE_NOPAYLOAD_12`| 3                           |3               | -                  | -                  |  12 bits cover all in-page transitions    | 
+|`SPARSE_12`          | 2 + _CC_ * 2.5              |5               | 25                 | 27                 |  Size of sparse is proportional to number of children |
+|`DENSE_12`           | 3 + _CS_ * 1.5              |5               | 18                 | 140                |  Lookup in dense is faster, size smaller if few holes    | 
+|`SINGLE_16`          | 4                           |4               | -                  | -                  |                                |    
+|`SPARSE_16`          | 2 + _CC_ * 3                |5               | 29                 | 32                 |                                |     
+|`DENSE_16`           | 3 + _CS_ * 2                |5               | 23                 | 185                |                                |     
+|`SPARSE_24`          | 2 + _CC_ * 4                |6               | 38                 | 42                 |                                |     
+|`DENSE_24`           | 3 + _CS_ * 3                |6               | 33                 | 276                |                                |     
+|`DENSE_32`           | 3 + _CS_ * 4                |7               | 43                 | 367                |  Nodes with big subtrees are usually dense   | 
+|`SPARSE_40`          | 2 + _CC_ * 6                |8               | 56                 | 62                 |                                |     
+|`DENSE_40`           | 3 + _CS_ * 5                |8               | 53                 | 458                |                                |     
+|`DENSE_LONG`         | 3 + _CS_ * 8                |11              | 83                 | 731                |  Catch-all                     |
+
+All pointers are stored as distances, and since all tries are written
+from the bottom up (and hence a child is always before the parent in the
+file), the distance is subtracted from the position of the current node
+to obtain the position of the child node.
+
+Note: All nodes are placed in such a way that they do not cross a page
+boundary. I.e. if a reader (e.g. [`Walker`](../../../tries/Walker.java)) 
+is positioned at a node, it is
+guaranteed that all reads of the node's data can complete without
+requiring a different page to be fetched from disk.
+
+## Partition index
+Implemented in [`PartitionIndex.java`](PartitionIndex.java)
+
+Layout:
+```
+[nodes page, 4096 bytes]
+...
+[nodes page, 4096 bytes]
+[nodes page including root node, ≤4096 bytes]
+[smallest key, with short length]
+[largest key, with short length]
+[smallest key pos, long]
+[key count, long]
+[root pos, long]
+```
+
+The SSTable's partition index is stored in the -Partitions.db file. The
+file itself is written from the bottom up, and its "header" is at the
+end of the file.
+
+More precisely, the last 3 longs in the file contain:
+
+-   A file position where the smallest and greatest key are written.
+
+-   The exact number of keys in the file.
+
+-   A file position for the root node of the index.
+
+These three longs are preceded by the serialization of the first and
+last key, and before that are the trie contents.
+
+To find a match for the key, start at the root position, decode the node
+(see the "Trie nodes" section above) and follow the transitions
+according to the bytes of the byte-ordered representation of the key
+while the node has children and there are bytes left in the key.
+
+If a leaf node is reached, that node contains the following payload:
+
+-   If _pb_ < 8, let
+
+    -   _idxpos_ be the sign-extended integer value of length _pb_ at
+        _ppos_
+
+-   If _pb_ &ge; 8 (always the case in DSE 6 files), let

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178651494


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.

Review Comment:
   So the point here is that if we write exactly one block (or none), we don't need the trie, as we'd have to "slow scan" even the single block. Perhaps the comment could make that more clear, but that might just be me being dense :D



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167167136


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings("resource")
+            PartitionIndex pi = new PartitionIndexEarly(fh, partialIndexTail.root(), partialIndexTail.count(), firstKey, partialIndexLastKey, partialIndexTail.cutoff(), partialIndexTail.tail());
+            partialIndexConsumer.accept(pi);
+            partialIndexConsumer = null;
+        }
+        finally
+        {
+            fhBuilder.withLengthOverride(-1);
+        }
+
+    }
+
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexBuilder.class);

Review Comment:
   nit: unused logger



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1170661912


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+  
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size_in_kb`

Review Comment:
   ```suggestion
   _granularity_ below, specified by the `column_index_size`
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1169270828


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).

Review Comment:
   nit: Perhaps https://books.google.com/books?id=dVipBwAAQBAJ for an English-speaking audience?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178646591


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.
+            return -1;
+        }
+    }
+
+    protected void addIndexBlock() throws IOException
+    {
+        IndexInfo cIndexInfo = new IndexInfo(startPosition,

Review Comment:
   format/nit: Line break seems unnecessary



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167206170


##########
src/java/org/apache/cassandra/io/tries/TrieNode.java:
##########
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SizedInts;
+
+/**
+ * Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as
+ * they are on disk without any serialization, and to enable the creation of such files.
+ * <p>
+ * The serialization methods take as argument a generic {@code SerializationNode} and provide a method {@code typeFor}
+ * for choosing a suitable type to represent it, which can then be used to calculate size and write the node.
+ * <p>
+ * To read a file containing trie nodes, one would use {@code at} to identify the node type and then the various
+ * read methods to retrieve the data. They all take a buffer (usually memory-mapped) containing the data, and a position
+ * in it that identifies the node.
+ * <p>
+ * These node types do not specify any treatment of payloads. They are only concerned with providing 4 bits of
+ * space for {@code payloadFlags}, and a way of calculating the position after the node. Users of this class by convention
+ * use non-zero payloadFlags to indicate a payload exists, write it (possibly in flag-dependent format) at serialization
+ * time after the node itself is written, and read it using the {@code payloadPosition} value.
+ * <p>
+ * To improve efficiency, multiple node types depending on the number of transitions are provided:
+ * -- payload only, which has no outgoing transitions
+ * -- single outgoing transition
+ * -- sparse, which provides a list of transition bytes with corresponding targets
+ * -- dense, where the transitions span a range of values and having the list (and the search in it) can be avoided
+ * <p>
+ * For each of the transition-carrying types we also have "in-page" versions where transition targets are the 4, 8 or 12
+ * lowest bits of the position within the same page. To save one further byte, the single in-page versions using 4 or 12
+ * bits cannot carry a payload.
+ * <p>
+ * This class is effectively an enumeration; abstract class permits instances to extend each other and reuse code.
+ */
+public abstract class TrieNode
+{
+    // Consumption (read) methods
+
+    /**
+     * Returns the type of node stored at this position. It can then be used to call the methods below.
+     */
+    public static TrieNode at(ByteBuffer src, int position)
+    {
+        return values[(src.get(position) >> 4) & 0xF];
+    }
+
+    /**
+     * Returns the 4 payload flag bits. Node types that cannot carry a payload return 0.
+     */
+    public int payloadFlags(ByteBuffer src, int position)
+    {
+        return src.get(position) & 0x0F;
+    }
+
+    /**
+     * Return the position just after the node, where the payload is usually stored.
+     */
+    abstract public int payloadPosition(ByteBuffer src, int position);
+
+    /**
+     * Returns search index for the given byte in the node. If exact match is present, this is >= 0, otherwise as in
+     * binary search.
+     */
+    abstract public int search(ByteBuffer src, int position, int transitionByte);       // returns as binarySearch
+
+    /**
+     * Returns the upper childIndex limit. Calling transition with values 0...transitionRange - 1 is valid.
+     */
+    abstract public int transitionRange(ByteBuffer src, int position);
+
+    /**
+     * Returns the byte value for this child index, or Integer.MAX_VALUE if there are no transitions with this index or
+     * higher to permit listing the children without needing to call transitionRange.
+     *
+     * @param childIndex must be >= 0, though it is allowed to pass a value greater than {@code transitionRange - 1}
+     */
+    abstract public int transitionByte(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns the delta between the position of this node and the position of the target of the specified transition.
+     * This is always a negative number. Dense nodes use 0 to specify "no transition".
+     *
+     * @param childIndex must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                   and behaviour of this method is undefined for values outside of that range
+     */
+    abstract long transitionDelta(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns position of node to transition to for the given search index. Argument must be positive. May return -1
+     * if a transition with that index does not exist (DENSE nodes).
+     * Position is the offset of the node within the ByteBuffer. positionLong is its global placement, which is the
+     * base for any offset calculations.
+     *
+     * @param positionLong although it seems to be obvious, this argument must be "real", that is, each child must have
+     *                     the calculated absolute position >= 0, otherwise the behaviour of this method is undefined
+     * @param childIndex   must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                     and behaviour of this method is undefined for values outside of that range
+     */
+    public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+    {
+        // note: this is not valid for dense nodes
+        return positionLong + transitionDelta(src, position, childIndex);
+    }
+
+    /**
+     * Returns the highest transition for this node, or -1 if none exist (PAYLOAD_ONLY nodes).
+     */
+    public long lastTransition(ByteBuffer src, int position, long positionLong)
+    {
+        return transition(src, position, positionLong, transitionRange(src, position) - 1);
+    }
+
+    /**
+     * Returns a transition that is higher than the index returned by {@code search}. This may not exist (if the
+     * argument was higher than the last transition byte), in which case this returns the given {@code defaultValue}.
+     */
+    abstract public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    /**
+     * Returns a transition that is lower than the index returned by {@code search}. Returns {@code defaultValue} for
+     * {@code searchIndex} equals 0 or -1 as lesser transition for those indexes does not exist.
+     */
+    abstract public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    // Construction (serialization) methods
+
+    /**
+     * Returns a node type that is suitable to store the node.
+     */
+    public static TrieNode typeFor(SerializationNode<?> node, long nodePosition)
+    {
+        int c = node.childCount();
+        if (c == 0)
+            return PAYLOAD_ONLY;
+
+        int bitsPerPointerIndex = 0;
+        long delta = node.maxPositionDelta(nodePosition);
+        assert delta < 0;
+        while (!singles[bitsPerPointerIndex].fits(-delta))
+            ++bitsPerPointerIndex;
+
+        if (c == 1)
+        {
+            if (node.payload() != null && singles[bitsPerPointerIndex].bytesPerPointer == FRACTIONAL_BYTES)
+                ++bitsPerPointerIndex; // next index will permit payload
+
+            return singles[bitsPerPointerIndex];
+        }
+
+        TrieNode sparse = sparses[bitsPerPointerIndex];
+        TrieNode dense = denses[bitsPerPointerIndex];
+        return (sparse.sizeofNode(node) < dense.sizeofNode(node)) ? sparse : dense;
+    }
+
+    /**
+     * Returns the size needed to serialize this node.
+     */
+    abstract public int sizeofNode(SerializationNode<?> node);
+
+    /**
+     * Serializes the node. All transition target positions must already have been defined. {@code payloadBits} must
+     * be four bits.
+     */
+    abstract public void serialize(DataOutputPlus out, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException;
+
+    // Implementations
+
+    final int bytesPerPointer;
+    static final int FRACTIONAL_BYTES = 0;
+
+    TrieNode(int ordinal, int bytesPerPointer)
+    {
+        this.ordinal = ordinal;
+        this.bytesPerPointer = bytesPerPointer;
+    }
+
+    final int ordinal;
+
+    static final TrieNode PAYLOAD_ONLY = new PayloadOnly();
+
+    static private class PayloadOnly extends TrieNode
+    {
+        // byte flags
+        // var payload
+        PayloadOnly()
+        {
+            super(0, FRACTIONAL_BYTES);
+        }
+
+        @Override
+        public int payloadPosition(ByteBuffer src, int position)
+        {
+            return position + 1;
+        }
+
+        @Override
+        public int search(ByteBuffer src, int position, int transitionByte)
+        {
+            return -1;
+        }
+
+        @Override
+        public long transitionDelta(ByteBuffer src, int position, int childIndex)
+        {
+            return 0;
+        }
+
+        @Override
+        public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+        {
+            return -1;
+        }
+
+        @Override
+        public long lastTransition(ByteBuffer src, int position, long positionLong)
+        {
+            return -1;
+        }
+
+        @Override
+        public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public int transitionByte(ByteBuffer src, int position, int childIndex)
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public int transitionRange(ByteBuffer src, int position)
+        {
+            return 0;
+        }
+
+        public int sizeofNode(SerializationNode<?> node)
+        {
+            return 1;
+        }
+
+        @Override
+        public void serialize(DataOutputPlus dest, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException
+        {
+            dest.writeByte((ordinal << 4) + (payloadBits & 0x0F));
+        }
+    }
+
+    static final TrieNode SINGLE_8 = new Single(2, 1);
+    static final TrieNode SINGLE_16 = new Single(4, 2);

Review Comment:
   Aside: I think we moved these into `Single` itself in one of the SAI patches to avoid potential class loading deadlocks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167207425


##########
src/java/org/apache/cassandra/io/tries/TrieNode.java:
##########
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SizedInts;
+
+/**
+ * Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as
+ * they are on disk without any serialization, and to enable the creation of such files.
+ * <p>
+ * The serialization methods take as argument a generic {@code SerializationNode} and provide a method {@code typeFor}
+ * for choosing a suitable type to represent it, which can then be used to calculate size and write the node.
+ * <p>
+ * To read a file containing trie nodes, one would use {@code at} to identify the node type and then the various
+ * read methods to retrieve the data. They all take a buffer (usually memory-mapped) containing the data, and a position
+ * in it that identifies the node.
+ * <p>
+ * These node types do not specify any treatment of payloads. They are only concerned with providing 4 bits of
+ * space for {@code payloadFlags}, and a way of calculating the position after the node. Users of this class by convention
+ * use non-zero payloadFlags to indicate a payload exists, write it (possibly in flag-dependent format) at serialization
+ * time after the node itself is written, and read it using the {@code payloadPosition} value.
+ * <p>
+ * To improve efficiency, multiple node types depending on the number of transitions are provided:
+ * -- payload only, which has no outgoing transitions
+ * -- single outgoing transition
+ * -- sparse, which provides a list of transition bytes with corresponding targets
+ * -- dense, where the transitions span a range of values and having the list (and the search in it) can be avoided
+ * <p>
+ * For each of the transition-carrying types we also have "in-page" versions where transition targets are the 4, 8 or 12
+ * lowest bits of the position within the same page. To save one further byte, the single in-page versions using 4 or 12
+ * bits cannot carry a payload.
+ * <p>
+ * This class is effectively an enumeration; abstract class permits instances to extend each other and reuse code.
+ */
+public abstract class TrieNode
+{
+    // Consumption (read) methods
+
+    /**
+     * Returns the type of node stored at this position. It can then be used to call the methods below.
+     */
+    public static TrieNode at(ByteBuffer src, int position)
+    {
+        return values[(src.get(position) >> 4) & 0xF];
+    }
+
+    /**
+     * Returns the 4 payload flag bits. Node types that cannot carry a payload return 0.
+     */
+    public int payloadFlags(ByteBuffer src, int position)
+    {
+        return src.get(position) & 0x0F;
+    }
+
+    /**
+     * Return the position just after the node, where the payload is usually stored.
+     */
+    abstract public int payloadPosition(ByteBuffer src, int position);
+
+    /**
+     * Returns search index for the given byte in the node. If exact match is present, this is >= 0, otherwise as in
+     * binary search.
+     */
+    abstract public int search(ByteBuffer src, int position, int transitionByte);       // returns as binarySearch
+
+    /**
+     * Returns the upper childIndex limit. Calling transition with values 0...transitionRange - 1 is valid.
+     */
+    abstract public int transitionRange(ByteBuffer src, int position);
+
+    /**
+     * Returns the byte value for this child index, or Integer.MAX_VALUE if there are no transitions with this index or
+     * higher to permit listing the children without needing to call transitionRange.
+     *
+     * @param childIndex must be >= 0, though it is allowed to pass a value greater than {@code transitionRange - 1}
+     */
+    abstract public int transitionByte(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns the delta between the position of this node and the position of the target of the specified transition.
+     * This is always a negative number. Dense nodes use 0 to specify "no transition".
+     *
+     * @param childIndex must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                   and behaviour of this method is undefined for values outside of that range
+     */
+    abstract long transitionDelta(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns position of node to transition to for the given search index. Argument must be positive. May return -1
+     * if a transition with that index does not exist (DENSE nodes).
+     * Position is the offset of the node within the ByteBuffer. positionLong is its global placement, which is the
+     * base for any offset calculations.
+     *
+     * @param positionLong although it seems to be obvious, this argument must be "real", that is, each child must have
+     *                     the calculated absolute position >= 0, otherwise the behaviour of this method is undefined
+     * @param childIndex   must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                     and behaviour of this method is undefined for values outside of that range
+     */
+    public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+    {
+        // note: this is not valid for dense nodes
+        return positionLong + transitionDelta(src, position, childIndex);
+    }
+
+    /**
+     * Returns the highest transition for this node, or -1 if none exist (PAYLOAD_ONLY nodes).
+     */
+    public long lastTransition(ByteBuffer src, int position, long positionLong)
+    {
+        return transition(src, position, positionLong, transitionRange(src, position) - 1);
+    }
+
+    /**
+     * Returns a transition that is higher than the index returned by {@code search}. This may not exist (if the
+     * argument was higher than the last transition byte), in which case this returns the given {@code defaultValue}.
+     */
+    abstract public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    /**
+     * Returns a transition that is lower than the index returned by {@code search}. Returns {@code defaultValue} for
+     * {@code searchIndex} equals 0 or -1 as lesser transition for those indexes does not exist.
+     */
+    abstract public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    // Construction (serialization) methods
+
+    /**
+     * Returns a node type that is suitable to store the node.
+     */
+    public static TrieNode typeFor(SerializationNode<?> node, long nodePosition)
+    {
+        int c = node.childCount();
+        if (c == 0)
+            return PAYLOAD_ONLY;
+
+        int bitsPerPointerIndex = 0;
+        long delta = node.maxPositionDelta(nodePosition);
+        assert delta < 0;
+        while (!singles[bitsPerPointerIndex].fits(-delta))
+            ++bitsPerPointerIndex;
+
+        if (c == 1)
+        {
+            if (node.payload() != null && singles[bitsPerPointerIndex].bytesPerPointer == FRACTIONAL_BYTES)
+                ++bitsPerPointerIndex; // next index will permit payload
+
+            return singles[bitsPerPointerIndex];
+        }
+
+        TrieNode sparse = sparses[bitsPerPointerIndex];
+        TrieNode dense = denses[bitsPerPointerIndex];
+        return (sparse.sizeofNode(node) < dense.sizeofNode(node)) ? sparse : dense;
+    }
+
+    /**
+     * Returns the size needed to serialize this node.
+     */
+    abstract public int sizeofNode(SerializationNode<?> node);
+
+    /**
+     * Serializes the node. All transition target positions must already have been defined. {@code payloadBits} must
+     * be four bits.
+     */
+    abstract public void serialize(DataOutputPlus out, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException;
+
+    // Implementations
+
+    final int bytesPerPointer;
+    static final int FRACTIONAL_BYTES = 0;
+
+    TrieNode(int ordinal, int bytesPerPointer)
+    {
+        this.ordinal = ordinal;
+        this.bytesPerPointer = bytesPerPointer;
+    }
+
+    final int ordinal;
+
+    static final TrieNode PAYLOAD_ONLY = new PayloadOnly();
+
+    static private class PayloadOnly extends TrieNode
+    {
+        // byte flags
+        // var payload
+        PayloadOnly()
+        {
+            super(0, FRACTIONAL_BYTES);
+        }
+
+        @Override
+        public int payloadPosition(ByteBuffer src, int position)
+        {
+            return position + 1;
+        }
+
+        @Override
+        public int search(ByteBuffer src, int position, int transitionByte)
+        {
+            return -1;
+        }
+
+        @Override
+        public long transitionDelta(ByteBuffer src, int position, int childIndex)
+        {
+            return 0;
+        }
+
+        @Override
+        public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+        {
+            return -1;
+        }
+
+        @Override
+        public long lastTransition(ByteBuffer src, int position, long positionLong)
+        {
+            return -1;
+        }
+
+        @Override
+        public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public int transitionByte(ByteBuffer src, int position, int childIndex)
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public int transitionRange(ByteBuffer src, int position)
+        {
+            return 0;
+        }
+
+        public int sizeofNode(SerializationNode<?> node)
+        {
+            return 1;
+        }
+
+        @Override
+        public void serialize(DataOutputPlus dest, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException
+        {
+            dest.writeByte((ordinal << 4) + (payloadBits & 0x0F));
+        }
+    }
+
+    static final TrieNode SINGLE_8 = new Single(2, 1);
+    static final TrieNode SINGLE_16 = new Single(4, 2);

Review Comment:
   See https://github.com/apache/cassandra/pull/1466/commits/b45f47d7097b2c9fbf6abe1c10ae60543b0068cb



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167251986


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173519304


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba

Review Comment:
   Line removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180769892


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;

Review Comment:
   Aside: So `size` is the number of bytes in the payload, but where does the `7` come from? (The hash is just one byte?)
   
   
   EDIT: In any case, an appropriately named constant for the several places we use `7` would probably be helpful.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181886672


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {

Review Comment:
   nit: Any tests that hit this branch (i.e. when there's an actual row index)?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181869738


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);

Review Comment:
   nit: Unrelated to this patch I guess, but `SSTableReader`'s `notifySkipped()` and `notifySelected()` methods ignore `updateStats`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181735536


##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.

Review Comment:
   nit: Worth mentioning it might also just be directly pointing to a position in the data file if there's no row index? Documentation is probably enough to manage the overloading, although maybe it would be nice to have a non-row-indexed `AbstractRowIndexEntry`? (Perhaps it would make things like the assertion in `serialize()` below less necessary...)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180811127


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException

Review Comment:
   nit: Method used only in test code? (i.e. in `PartitionIndexTest`)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179751508


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).

Review Comment:
   nit: Remove DB reference.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183991926


##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableVerifier.java:
##########
@@ -140,6 +142,33 @@ protected void markAndThrow(Throwable cause, boolean mutateRepaired)
             throw new RuntimeException(e);
     }
 
+    public void verify()

Review Comment:
   Are you able to run `org.apache.cassandra.tools.nodetool.VerifyTest` w/ J11 from IDEA?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182984540


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;

Review Comment:
   nit: Not a big deal, and it might actually be more future proof this way, but `dataSizeFromIndex` is already guaranteed to be `-1` if we get here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183349069


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");

Review Comment:
   The scrub output does contain the sstable name, at the very least in the completion messages done by `SortedTableScrubber.outputSummary`.
   Adding the context here will likely make the output less readable.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;

Review Comment:
   I prefer to reset all.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;
+                    }
+                }
+            }
+
+            String keyName = key == null ? "(unreadable key)" : keyString(key);
+            outputHandler.debug("partition %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSizeFromIndex));
+
+            try
+            {
+                if (key == null)
+                    throw new IOError(new IOException("Unable to read partition key from data file", keyReadError));
+
+                if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
+                {
+                    throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
+                                                                    //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
+                                                                    "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
+                }
+
+                if (indexIterator != null && dataSizeFromIndex > dataFile.length())
+                    throw new IOError(new IOException("Impossible partition size (greater than file length): " + dataSizeFromIndex));
+
+                if (indexIterator != null && dataStart != dataStartFromIndex)
+                    outputHandler.warn("Data file partition position %d differs from index file row position %d", dataStart, dataStartFromIndex);
+
+                if (tryAppend(prevKey, key, writer))
+                    prevKey = key;
+            }
+            catch (Throwable th)
+            {
+                throwIfFatal(th);
+                outputHandler.warn(th, "Error reading partition %s (stacktrace follows):", keyName);
+
+                if (currentIndexKey != null
+                    && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex))
+                {
+
+                    // position where the row should start in a data file (right after the partition key)
+                    long rowStartFromIndex = dataStartFromIndex + TypeSizes.SHORT_SIZE + currentIndexKey.remaining();
+                    outputHandler.output("Retrying from partition index; data is %s bytes starting at %s",
+                                         dataSizeFromIndex, rowStartFromIndex);
+                    key = sstable.decorateKey(currentIndexKey);
+                    try
+                    {
+                        if (!cfs.metadata.getLocal().isIndex())
+                            cfs.metadata.getLocal().partitionKeyType.validate(key.getKey());
+                        dataFile.seek(rowStartFromIndex);
+
+                        if (tryAppend(prevKey, key, writer))
+                            prevKey = key;
+                    }
+                    catch (Throwable th2)
+                    {
+                        throwIfFatal(th2);
+                        throwIfCannotContinue(key, th2);
+
+                        outputHandler.warn(th2, "Retry failed too. Skipping to next partition (retry's stacktrace follows)");
+                        badPartitions++;
+                        if (!seekToNextPartition())
+                            break;
+                    }
+                }
+                else
+                {
+                    throwIfCannotContinue(key, th);
+
+                    badPartitions++;
+                    if (indexIterator != null)
+                    {
+                        outputHandler.warn("Partition starting at position %d is unreadable; skipping to next", dataStart);
+                        if (!seekToNextPartition())
+                            break;
+                    }
+                    else
+                    {
+                        outputHandler.warn("Unrecoverable error while scrubbing %s." +
+                                           "Scrubbing cannot continue. The sstable will be marked for deletion. " +
+                                           "You can attempt manual recovery from the pre-scrub snapshot. " +
+                                           "You can also run nodetool repair to transfer the data from a healthy replica, if any.",
+                                           sstable);
+                        // There's no way to resync and continue. Give up.
+                        break;
+                    }
+                }
+            }
+        }
+    }
+
+
+    private boolean indexAvailable()
+    {
+        return indexIterator != null && !indexIterator.isExhausted();
+    }
+
+    private boolean seekToNextPartition()
+    {
+        while (indexAvailable())
+        {
+            long nextRowPositionFromIndex = indexIterator.dataPosition();
+
+            try
+            {
+                dataFile.seek(nextRowPositionFromIndex);
+                return true;
+            }
+            catch (Throwable th)
+            {
+                throwIfFatal(th);
+                outputHandler.warn(th, "Failed to seek to next row position %d", nextRowPositionFromIndex);
+                badPartitions++;
+            }
+
+            try
+            {
+                indexIterator.advance();
+            }
+            catch (Throwable th)
+            {
+                outputHandler.warn(th, "Failed to go to the next entry in index");
+                throw Throwables.cleaned(th);
+            }
+        }
+
+        return false;
+    }
+
+    @Override
+    protected void throwIfCannotContinue(DecoratedKey key, Throwable th)
+    {
+        if (isIndex)
+        {
+            outputHandler.warn("An error occurred while scrubbing the partition with key '%s' for an index table. " +
+                               "Scrubbing will abort for this table and the index will be rebuilt.", keyString(key));
+            throw new IOError(th);
+        }
+
+        super.throwIfCannotContinue(key, th);
+    }
+
+    public void close()

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();

Review Comment:
   Yes. First and last entry need to be rechecked against the boundaries, and we do the latter by reading one entry ahead.
   
   Added class JavaDoc.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();

Review Comment:
   Added message and also changed to continue without index in this case.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())

Review Comment:
   Changed to `isIndex` (`metadata.get()` and `getLocal()` can only be different in details like compression) and cached the partition key type as well.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ * <p>
+ * The difference between this and {@Link PartitionIterator} is that this only uses information present in the index file

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ * <p>
+ * The difference between this and {@Link PartitionIterator} is that this only uses information present in the index file
+ * and does not try to read keys of the data file (for the trie index format), thus {@link #key()} can be null.
+ * <p>
+ * Starts advanced to a position, {@link #advance()} is to be used to go to next, and iteration completes when
+ * {@link #dataPosition()} == -1.
+ */
+public interface ScrubPartitionIterator extends Closeable
+{
+    /**
+     * Serialized partition key or {@code null} if the iterator reached the end of the index or if the key may not
+     * be fully retrieved from the index file.
+     * @return

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th,
+                                           "Failed to advance to the next index position. Index is corrupted. " +
+                                           "Continuing without the index. Last position read is %d.",
+                                           indexIterator.dataPosition());
+                        indexIterator.close();
+                        indexIterator = null;
+                        currentIndexKey = null;
+                        dataStartFromIndex = -1;
+                        dataSizeFromIndex = -1;
+                    }
+                }
+            }
+
+            String keyName = key == null ? "(unreadable key)" : keyString(key);
+            outputHandler.debug("partition %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSizeFromIndex));
+
+            try
+            {
+                if (key == null)
+                    throw new IOError(new IOException("Unable to read partition key from data file", keyReadError));
+
+                if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
+                {
+                    throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
+                                                                    //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));

Review Comment:
   Hm... this was introduced in CASSANDRA-8099 and is probably an oversight there.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubIterator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ScrubIterator extends PartitionIndex.IndexPosIterator implements ScrubPartitionIterator
+{
+    ByteBuffer key;
+    long dataPosition;
+    final FileHandle rowIndexFile;
+
+    ScrubIterator(PartitionIndex partitionIndex, FileHandle rowIndexFile) throws IOException
+    {
+        super(partitionIndex);
+        this.rowIndexFile = rowIndexFile.sharedCopy();
+        advance();
+    }
+
+    @Override
+    public void close()
+    {
+        super.close();
+        rowIndexFile.close();
+    }
+
+    @Override
+    public ByteBuffer key()
+    {
+        return key;
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return dataPosition;
+    }
+
+    @Override
+    public void advance() throws IOException
+    {
+        long pos = nextIndexPos();
+        if (pos != PartitionIndex.NOT_FOUND)
+        {
+            if (pos >= 0) // row index position
+            {
+                try (FileDataInput in = rowIndexFile.createReader(pos))
+                {
+                    key = ByteBufferUtil.readWithShortLength(in);
+                    dataPosition = TrieIndexEntry.deserialize(in, in.getFilePointer()).position;
+                }
+            }
+            else
+            {
+                key = null;
+                dataPosition = ~pos;
+            }
+        }
+        else
+        {
+            key = null;
+            dataPosition = -1;
+        }
+    }
+
+    @Override
+    public boolean isExhausted()
+    {
+        return dataPosition == -1;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();
+            currentPosition = file.getFilePointer();

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182078234


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            checkArgument(!online || builder.getSerializationHeader() != null);
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings({ "resource", "RedundantSuppression" })
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+                    FilterComponent.save(filter, descriptor, false);
+                }
+            }
+
+            if (builder.getFilter() == null)
+                builder.setFilter(FilterFactory.AlwaysPresent);
+
+            if (builder.getComponents().contains(Components.ROW_INDEX))
+                builder.setRowIndexFile(rowIndexFileBuilder().complete());
+
+            if (descriptor.version.hasKeyRange() && builder.getStatsMetadata() != null)
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX))
+            {
+                builder.setPartitionIndex(openPartitionIndex(builder.getFilter().isInformative()));
+                if (builder.getFirst() == null || builder.getLast() == null)
+                {
+                    builder.setFirst(builder.getPartitionIndex().firstKey());
+                    builder.setLast(builder.getPartitionIndex().lastKey());
+                }
+            }
+
+            try (CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components))
+            {
+                builder.setDataFile(dataFileBuilder(builder.getStatsMetadata()).withCompressionMetadata(compressionMetadata).complete());
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            // in case of failure, close only those components which have been opened in this try-catch block
+            Throwables.closeAndAddSuppressed(ex, builder.getPartitionIndex(), builder.getRowIndexFile(), builder.getDataFile(), builder.getFilter());
+            throw ex;
+        }
+    }
+
+    private IFilter buildBloomFilter(StatsMetadata statsMetadata) throws IOException
+    {
+        IFilter bf = null;
+
+        try (KeyReader keyReader = createKeyReader(statsMetadata))
+        {
+            bf = FilterFactory.getFilter(statsMetadata.totalRows, tableMetadataRef.getLocal().params.bloomFilterFpChance);
+
+            while (!keyReader.isExhausted())
+            {
+                DecoratedKey key = tableMetadataRef.getLocal().partitioner.decorateKey(keyReader.key());
+                bf.add(key);
+
+                keyReader.advance();
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            Throwables.closeAndAddSuppressed(ex, bf);
+            throw ex;
+        }
+
+        return bf;
+    }
+
+    private IFilter loadFilter(ValidationMetadata validationMetadata)
+    {
+        return FilterComponent.maybeLoadBloomFilter(descriptor,
+                                                    components,
+                                                    tableMetadataRef.get(),
+                                                    validationMetadata);
+    }
+
+    private PartitionIndex openPartitionIndex(boolean preload) throws IOException
+    {
+        try (FileHandle indexFile = partitionIndexFileBuilder().complete())
+        {
+            return PartitionIndex.load(indexFile, tableMetadataRef.getLocal().partitioner, preload);
+        }
+        catch (IOException ex)
+        {
+            logger.debug("Partition index file is corrupted: " + descriptor.fileFor(Components.PARTITION_INDEX), ex);
+            throw ex;
+        }
+    }
+
+    private FileHandle.Builder dataFileBuilder(StatsMetadata statsMetadata)

Review Comment:
   This method looks duplicated exactly in `BigSSTableReaderLoadingBuilder`, so I'm guessing we could move it into `SSTableReaderLoadingBuilder` (along w/ the `dataFileBuilder` field, which isn't `final`).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183003370


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183957196


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        @Override
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        @Override
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        @Override
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;

Review Comment:
   nit: Brief message?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1184168749


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ *   - calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   - delaying writing any part of a completed node until its branch size is above the page size
+ *   - laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   - adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ *
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ *
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * where lines denote page boundaries.
+ *
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;

Review Comment:
   nit: Any tests where `hasOutOfPageChildren == true`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185509973


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException
+    {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        // Check that long repeated strings don't cause stack overflow
+        // Test both normal completion and partial construction.
+        Thread t = new Thread(null, () ->
+        {
+            try
+            {
+                File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+                SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build());
+                List<DecoratedKey> list = Lists.newArrayList();
+                String longString = "";
+                for (int i = 0; i < PageAware.PAGE_SIZE + 99; ++i)
+                {
+                    longString += i;
+                }
+                IPartitioner partitioner = ByteOrderedPartitioner.instance;
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "A")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "B")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "C")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "D")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "E")));
+
+                FileHandle.Builder fhBuilder = new FileHandle.Builder(file)
+                                               .bufferSize(PageAware.PAGE_SIZE)
+                                               .withChunkCache(ChunkCache.instance);
+                try (PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder))
+                {
+                    int i = 0;
+                    for (i = 0; i < 3; ++i)
+                        builder.addEntry(list.get(i), i);
+
+                    writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                    AtomicInteger callCount = new AtomicInteger();
+
+                    final int addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             index.close();
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+
+                    for (; i < list.size(); ++i)
+                        builder.addEntry(list.get(i), i);
+                    builder.complete();
+
+                    try (PartitionIndex index = PartitionIndex.load(fhBuilder, partitioner, true))
+                    {
+                        checkIteration(list, list.size(), index);
+                    }
+                }
+                future.complete(null);
+            }
+            catch (Throwable err)
+            {
+                future.completeExceptionally(err);
+            }
+        }, "testThread", 32 * 1024);
+
+        t.start();
+        future.join();
+    }
+
+    class JumpingFile extends SequentialWriter
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingFile(File file, SequentialWriterOption option, long... cutoffsAndOffsets)
+        {
+            super(file, option);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public long position()
+        {
+            return jumped(super.position(), cutoffs, offsets);
+        }
+    }
+
+    class JumpingRebufferer extends WrappingRebufferer
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingRebufferer(Rebufferer source, long... cutoffsAndOffsets)
+        {
+            super(source);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            long pos;
+
+            int idx = Arrays.binarySearch(offsets, position);
+            if (idx < 0)
+                idx = -2 - idx;
+            pos = position;
+            if (idx >= 0)
+                pos = pos - offsets[idx] + cutoffs[idx];
+
+            super.rebuffer(pos);
+
+            if (idx < cutoffs.length - 1 && buffer.limit() + offset > cutoffs[idx + 1])
+                buffer.limit((int) (cutoffs[idx + 1] - offset));
+            if (idx >= 0)
+                offset = offset - cutoffs[idx] + offsets[idx];
+
+            return this;
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return jumped(wrapped.fileLength(), cutoffs, offsets);
+        }
+
+        @Override
+        public String toString()
+        {
+            return Arrays.toString(cutoffs) + Arrays.toString(offsets);
+        }
+    }
+
+    public class PartitionIndexJumping extends PartitionIndex
+    {
+        final long[] cutoffsAndOffsets;
+
+        public PartitionIndexJumping(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last,
+                                     long... cutoffsAndOffsets)
+        {
+            super(fh, trieRoot, keyCount, first, last);
+            this.cutoffsAndOffsets = cutoffsAndOffsets;
+        }
+
+        @Override
+        protected Rebufferer instantiateRebufferer()
+        {
+            return new JumpingRebufferer(super.instantiateRebufferer(), cutoffsAndOffsets);
+        }
+    }
+
+    long jumped(long pos, long[] cutoffs, long[] offsets)
+    {
+        int idx = Arrays.binarySearch(cutoffs, pos);
+        if (idx < 0)
+            idx = -2 - idx;
+        if (idx < 0)
+            return pos;
+        return pos - cutoffs[idx] + offsets[idx];
+    }
+
+    @Test
+    public void testPointerGrowth() throws IOException

Review Comment:
   nit: unnecessary `throws`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185508683


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException

Review Comment:
   nit: Unnecessary `throws`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162888227


##########
src/java/org/apache/cassandra/db/rows/LazilyInitializedUnfilteredRowIterator.java:
##########
@@ -105,4 +105,4 @@ public void close()
         if (iterator != null)
             iterator.close();
     }
-}
+}

Review Comment:
   Reverted



##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries
+
+    @Override
+    public int maxBytesInPage()
+    {
+        return buffer.capacity();
+    }
+
+    @Override
+    public void padToPageBoundary() throws IOException

Review Comment:
   Method removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164675713


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())

Review Comment:
   nit: line 97 to 125 duplicated in `BigTableVerifier#verifySSTable()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185537424


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException
+    {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        // Check that long repeated strings don't cause stack overflow
+        // Test both normal completion and partial construction.
+        Thread t = new Thread(null, () ->
+        {
+            try
+            {
+                File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+                SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build());
+                List<DecoratedKey> list = Lists.newArrayList();
+                String longString = "";
+                for (int i = 0; i < PageAware.PAGE_SIZE + 99; ++i)
+                {
+                    longString += i;
+                }
+                IPartitioner partitioner = ByteOrderedPartitioner.instance;
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "A")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "B")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "C")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "D")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "E")));
+
+                FileHandle.Builder fhBuilder = new FileHandle.Builder(file)
+                                               .bufferSize(PageAware.PAGE_SIZE)
+                                               .withChunkCache(ChunkCache.instance);
+                try (PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder))
+                {
+                    int i = 0;
+                    for (i = 0; i < 3; ++i)
+                        builder.addEntry(list.get(i), i);
+
+                    writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                    AtomicInteger callCount = new AtomicInteger();
+
+                    final int addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             index.close();
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+
+                    for (; i < list.size(); ++i)
+                        builder.addEntry(list.get(i), i);
+                    builder.complete();
+
+                    try (PartitionIndex index = PartitionIndex.load(fhBuilder, partitioner, true))
+                    {
+                        checkIteration(list, list.size(), index);
+                    }
+                }
+                future.complete(null);
+            }
+            catch (Throwable err)
+            {
+                future.completeExceptionally(err);
+            }
+        }, "testThread", 32 * 1024);
+
+        t.start();
+        future.join();
+    }
+
+    class JumpingFile extends SequentialWriter
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingFile(File file, SequentialWriterOption option, long... cutoffsAndOffsets)
+        {
+            super(file, option);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public long position()
+        {
+            return jumped(super.position(), cutoffs, offsets);
+        }
+    }
+
+    class JumpingRebufferer extends WrappingRebufferer
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingRebufferer(Rebufferer source, long... cutoffsAndOffsets)
+        {
+            super(source);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            long pos;
+
+            int idx = Arrays.binarySearch(offsets, position);
+            if (idx < 0)
+                idx = -2 - idx;
+            pos = position;
+            if (idx >= 0)
+                pos = pos - offsets[idx] + cutoffs[idx];
+
+            super.rebuffer(pos);
+
+            if (idx < cutoffs.length - 1 && buffer.limit() + offset > cutoffs[idx + 1])
+                buffer.limit((int) (cutoffs[idx + 1] - offset));
+            if (idx >= 0)
+                offset = offset - cutoffs[idx] + offsets[idx];
+
+            return this;
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return jumped(wrapped.fileLength(), cutoffs, offsets);
+        }
+
+        @Override
+        public String toString()
+        {
+            return Arrays.toString(cutoffs) + Arrays.toString(offsets);
+        }
+    }
+
+    public class PartitionIndexJumping extends PartitionIndex
+    {
+        final long[] cutoffsAndOffsets;
+
+        public PartitionIndexJumping(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last,
+                                     long... cutoffsAndOffsets)
+        {
+            super(fh, trieRoot, keyCount, first, last);
+            this.cutoffsAndOffsets = cutoffsAndOffsets;
+        }
+
+        @Override
+        protected Rebufferer instantiateRebufferer()
+        {
+            return new JumpingRebufferer(super.instantiateRebufferer(), cutoffsAndOffsets);
+        }
+    }
+
+    long jumped(long pos, long[] cutoffs, long[] offsets)
+    {
+        int idx = Arrays.binarySearch(cutoffs, pos);
+        if (idx < 0)
+            idx = -2 - idx;
+        if (idx < 0)
+            return pos;
+        return pos - cutoffs[idx] + offsets[idx];
+    }
+
+    @Test
+    public void testPointerGrowth() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            long[] cutoffsAndOffsets = new long[]{
+            2 * 4096, 1L << 16,
+            4 * 4096, 1L << 24,
+            6 * 4096, 1L << 31,
+            8 * 4096, 1L << 32,
+            10 * 4096, 1L << 33,
+            12 * 4096, 1L << 34,
+            14 * 4096, 1L << 40,
+            16 * 4096, 1L << 42
+            };
+
+            List<DecoratedKey> list = Lists.newArrayList();
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeJumpingWriter(file, cutoffsAndOffsets);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+
+                for (int i = 0; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                long root = builder.complete();
+
+                try (FileHandle fh = fhBuilder.complete();
+                     PartitionIndex index = new PartitionIndexJumping(fh, root, COUNT, null, null, cutoffsAndOffsets);
+                     Analyzer analyzer = new Analyzer(index))
+                {
+                    checkIteration(list, list.size(), index);
+
+                    analyzer.run();
+                    if (analyzer.countPerType.elementSet().size() < 7)
+                    {
+                        Assert.fail("Expecting at least 7 different node types, got " + analyzer.countPerType.elementSet().size() + "\n" + analyzer.countPerType);
+                    }
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDumpTrieToFile() throws IOException
+    {
+        File file = FileUtils.createTempFile("testDumpTrieToFile", "index");
+
+        ArrayList<DecoratedKey> list = Lists.newArrayList();
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.DEFAULT);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+            for (int i = 0; i < 1000; i++)
+            {
+                DecoratedKey key = generateRandomKey();
+                list.add(key);
+            }
+            Collections.sort(list);
+
+            for (int i = 0; i < 1000; ++i)
+                builder.addEntry(list.get(i), i);
+            long root = builder.complete();
+
+            try (FileHandle fh = fhBuilder.complete();
+                 PartitionIndex index = new PartitionIndex(fh, root, 1000, null, null))
+            {
+                File dump = FileUtils.createTempFile("testDumpTrieToFile", "dumpedTrie");
+                index.dumpTrie(dump.toString());
+                String dumpContent = String.join("\n", Files.readAllLines(dump.toPath()));
+                logger.info("Dumped trie: \n{}", dumpContent);
+                assertFalse(dumpContent.isEmpty());
+            }
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static class Analyzer extends PartitionIndex.Reader
+    {
+        Multiset<TrieNode> countPerType = HashMultiset.create();
+
+        public Analyzer(PartitionIndex index)
+        {
+            super(index);
+        }
+
+        public void run()
+        {
+            run(root);
+        }
+
+        void run(long node)
+        {
+            go(node);
+
+            countPerType.add(nodeType);
+
+            int tr = transitionRange();
+            for (int i = 0; i < tr; ++i)
+            {
+                long child = transition(i);
+                if (child == NONE)
+                    continue;
+                run(child);
+                go(node);
+            }
+        }
+    }
+
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateLongKeysIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateLongKey);
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateSequentialIndex(int size) throws IOException
+    {
+        return generateIndex(size, new Supplier<DecoratedKey>()
+        {
+            long i = 0;
+
+            public DecoratedKey get()
+            {
+                return sequentialKey(i++);
+            }
+        });
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndexWithZeroCopy(int size, int firstKeyOffset, int lastKeyOffset) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey, firstKeyOffset, lastKeyOffset, true);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator) throws IOException
+    {
+        return generateIndex(size, keyGenerator, 0, size - 1, false);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator, int firstKeyOffset, int lastKeyOffset, boolean hasZeroCopy) throws IOException

Review Comment:
   nit: `hasZeroCopy` unused, `throws` unnecessary



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185535689


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);

Review Comment:
   nit: Could replace all this w/
   
   ```
   assertThatThrownBy(() -> testGetEq(data)).isInstanceOfAny(AssertionError.class, IndexOutOfBoundsException.class);
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1503036052

   One more question , is it possible that one node have two sstable format (BIG and BIT)under one table ? if it is possible , and how is the compact done? BIT and BIG doing compaction for their own ? or mixed together ? and I think if it is possible ,we should add some compaction test case for this 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1503455415

   > One more question , is it possible that one node have two sstable format (BIG and BIT)under one table ?
   
   This is possible (e.g. after an upgrade where `upgradesstables` was not applied). The inputs are independently read (each by its format), and the output is created in the current write format.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181890536


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight)
+    {
+        super(partitionIndex, left, right);
+        this.partitionIndex = partitionIndex;
+        this.partitioner = partitioner;
+        this.limit = right;
+        this.exclusiveLimit = exclusiveRight;
+        this.rowIndexFile = rowIndexFile;
+        this.dataFile = dataFile;
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex)
+    {
+        super(partitionIndex, partitionIndex.firstKey(), partitionIndex.firstKey());
+        this.partitionIndex = partitionIndex;
+        this.partitioner = null;
+        this.limit = partitionIndex.firstKey();
+        this.exclusiveLimit = -1;
+        this.rowIndexFile = null;
+        this.dataFile = null;
+
+        this.currentEntry = null;
+        this.currentKey = null;
+        this.nextEntry = null;
+        this.nextKey = null;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accum = null;
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(partitionIndex, dataFile, rowIndexFile));
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(dataInput, indexInput));
+        accum = Throwables.perform(accum, super::close);
+        Throwables.maybeFail(accum);
+    }
+
+    public DecoratedKey decoratedKey()
+    {
+        return currentKey;
+    }
+
+    public ByteBuffer key()
+    {
+        return currentKey.getKey();
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return currentEntry != null ? currentEntry.position : -1;
+    }
+
+    @Override
+    public long keyPositionForSecondaryIndex()
+    {
+        return dataPosition();
+    }
+
+    public TrieIndexEntry entry()
+    {
+        return currentEntry;
+    }
+
+    @Override
+    public boolean advance() throws IOException
+    {
+        currentKey = nextKey;
+        currentEntry = nextEntry;
+        if (currentKey != null)
+        {
+            readNext();
+            // if nextKey is null, then currentKey is the last key to be published, therefore check against any limit
+            // and suppress the partition if it is beyond the limit
+            if (nextKey == null && limit != null && currentKey.compareTo(limit) > exclusiveLimit)
+            {   // exclude last partition outside range
+                currentKey = null;
+                currentEntry = null;
+                return false;
+            }
+            return true;
+        }
+        return false;
+    }
+
+    private void readNext() throws IOException
+    {
+        long pos = nextIndexPos();
+        if (pos != PartitionIndex.NOT_FOUND)
+        {
+            if (pos >= 0)
+            {
+                seekIndexInput(pos);
+                nextKey = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(indexInput));
+                nextEntry = TrieIndexEntry.deserialize(indexInput, indexInput.getFilePointer());
+            }
+            else
+            {
+                pos = ~pos;
+                seekDataInput(pos);
+                nextKey = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(dataInput));
+                nextEntry = new TrieIndexEntry(pos);
+            }
+        }
+        else
+        {
+            nextKey = null;
+            nextEntry = null;
+        }
+    }
+
+    private void seekIndexInput(long pos) throws IOException
+    {
+        if (indexInput == null)
+            indexInput = rowIndexFile.createReader(pos);
+        else
+            indexInput.seek(pos);
+    }
+
+    private void seekDataInput(long pos) throws IOException
+    {
+        if (dataInput == null)
+            dataInput = dataFile.createReader(pos);
+        else
+            dataInput.seek(pos);
+    }
+
+    @Override
+    public boolean isExhausted()
+    {
+        return currentKey == null;
+    }
+
+    @Override
+    public void reset()
+    {
+        go(root);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("TrieIndex-PartitionIndexIterator(%s)", partitionIndex.getFileHandle().path());

Review Comment:
   nit: "PartitionIterator" or "PartitionIndexIterator"?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185330646


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ * <ul>
+ *   <li> calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   <li> delaying writing any part of a completed node until its branch size is above the page size
+ *   <li> laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   <li> adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ * </ul>
+ * <p>
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ * <p>
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * <pre>
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * </pre>
+ * where lines denote page boundaries.
+ * <p>
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * <p>
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * <p>
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException

Review Comment:
   nit: @SuppressWarnings("DuplicatedCode")



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185460303


##########
test/unit/org/apache/cassandra/io/tries/AbstractTrieTestBase.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.function.BiFunction;
+
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+@RunWith(Parameterized.class)
+abstract public class AbstractTrieTestBase
+{
+    @Parameterized.Parameter(0)
+    public TestClass writerClass;
+
+    enum TestClass
+    {
+        SIMPLE(IncrementalTrieWriterSimple::new),
+        PAGE_AWARE(IncrementalTrieWriterPageAware::new),
+        PAGE_AWARE_DEEP_ON_STACK((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 256)),
+        PAGE_AWARE_DEEP_ON_HEAP((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 0)),
+        PAGE_AWARE_DEEP_MIXED((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 2));
+
+        final BiFunction<TrieSerializer<Integer, DataOutputPlus>, DataOutputPlus, IncrementalTrieWriter<Integer>> constructor;
+        TestClass(BiFunction<TrieSerializer<Integer, DataOutputPlus>, DataOutputPlus, IncrementalTrieWriter<Integer>> constructor)
+        {
+            this.constructor = constructor;
+        }
+    }
+
+    @Parameterized.Parameters(name = "{index}: trie writer class={0}")
+    public static Collection<Object[]> data()
+    {
+        return Arrays.asList(new Object[]{ TestClass.SIMPLE },
+                             new Object[]{ TestClass.PAGE_AWARE },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_ON_STACK },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_ON_HEAP },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_MIXED });
+    }
+
+    protected final static Logger logger = LoggerFactory.getLogger(TrieBuilderTest.class);
+    protected final static int BASE = 80;
+
+    protected boolean dump = false;
+    protected int payloadSize = 0;
+
+    @Before
+    public void beforeTest()
+    {
+        dump = false;
+        payloadSize = 0;
+    }
+
+    IncrementalTrieWriter<Integer> newTrieWriter(TrieSerializer<Integer, DataOutputPlus> serializer, DataOutputPlus out)
+    {
+        return writerClass.constructor.apply(serializer, out);
+    }
+
+    protected final TrieSerializer<Integer, DataOutputPlus> serializer = new TrieSerializer<Integer, DataOutputPlus>()
+    {
+        public int sizeofNode(SerializationNode<Integer> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + payloadSize;
+        }
+
+        public void write(DataOutputPlus dataOutput, SerializationNode<Integer> node, long nodePosition) throws IOException
+        {
+            if (dump)
+                logger.info("Writing at {} type {} size {}: {}", Long.toHexString(nodePosition), TrieNode.typeFor(node, nodePosition), TrieNode.typeFor(node, nodePosition).sizeofNode(node), node);
+            TrieNode.typeFor(node, nodePosition).serialize(dataOutput, node, node.payload() != null ? node.payload() : 0, nodePosition);

Review Comment:
   So just to confirm, the test serializer here is actually storing the payload in the payload flags?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183074110


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight)
+    {
+        super(partitionIndex, left, right);
+        this.partitionIndex = partitionIndex;
+        this.partitioner = partitioner;
+        this.limit = right;
+        this.exclusiveLimit = exclusiveRight;
+        this.rowIndexFile = rowIndexFile;
+        this.dataFile = dataFile;
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex)
+    {
+        super(partitionIndex, partitionIndex.firstKey(), partitionIndex.firstKey());
+        this.partitionIndex = partitionIndex;
+        this.partitioner = null;
+        this.limit = partitionIndex.firstKey();
+        this.exclusiveLimit = -1;
+        this.rowIndexFile = null;
+        this.dataFile = null;
+
+        this.currentEntry = null;
+        this.currentKey = null;
+        this.nextEntry = null;
+        this.nextKey = null;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accum = null;
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(partitionIndex, dataFile, rowIndexFile));
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(dataInput, indexInput));
+        accum = Throwables.perform(accum, super::close);
+        Throwables.maybeFail(accum);
+    }
+
+    public DecoratedKey decoratedKey()
+    {
+        return currentKey;
+    }
+
+    public ByteBuffer key()
+    {
+        return currentKey.getKey();
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return currentEntry != null ? currentEntry.position : -1;
+    }
+
+    @Override
+    public long keyPositionForSecondaryIndex()

Review Comment:
   Aside: SAI isn't even using this anymore, so I guess it'll be nice when we can remove SASI from the codebase...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185289232


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException

Review Comment:
   nit: Seems like the duplication is intended, so maybe throw a `@SuppressWarnings("DuplicatedCode")` on there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185290014


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException
+        {
+            nodePosition = nodePosition + node.branchSize;
+            assert dest.position() == nodePosition
+                    : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+            serializer.write(dest, node, nodePosition);
+
+            assert dest.position() == nodePosition + node.nodeSize
+                   || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                    : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+
+            node.filePos = nodePosition;
+        }
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException

Review Comment:
   nit: Seems like the duplication is intended, so maybe throw a `@SuppressWarnings("DuplicatedCode")` on there.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182287973


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;

Review Comment:
   Added



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ * <p>
+ * The files created by this builder are read by {@link PartitionIndex}.
+ */
+class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings({ "resource", "RedundantSuppression" })
+            PartitionIndex pi = new PartitionIndexEarly(fh, partialIndexTail.root(), partialIndexTail.count(), firstKey, partialIndexLastKey, partialIndexTail.cutoff(), partialIndexTail.tail());
+            partialIndexConsumer.accept(pi);
+            partialIndexConsumer = null;
+        }
+        finally
+        {
+            fhBuilder.withLengthOverride(-1);
+        }
+
+    }
+
+    /**
+    * @param decoratedKey the key for this record
+    * @param position the position to write with the record:
+    *    - positive if position points to an index entry in the index file
+    *    - negative if ~position points directly to the key in the data file
+    */
+    public PartitionIndexBuilder addEntry(DecoratedKey decoratedKey, long position) throws IOException

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;

Review Comment:
   Added a comment and a `FLAG_HAS_HAS_BITS` constant.
   
   



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Check for a prefix and find closest smaller branch.
+            Long indexPos = prefixAndNeighbours(key, Reader::getSpecificIndexPos);
+
+            if (indexPos != null && indexPos != NOT_FOUND)
+            {
+                ResultType res = acceptor.accept(indexPos, false, key);
+                if (res != null)
+                    return res;
+            }
+
+            // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+            // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+            // prefix match that is closer than max(lesserBranch).
+            if (lesserBranch == NONE)
+                return null;
+            goMax(lesserBranch);
+            indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public Long getSpecificIndexPos(int pos, int bits)
+        {
+            return getIndexPos(buf, pos, bits);
+        }
+
+        public long getCurrentIndexPos()
+        {
+            return getIndexPos(buf, payloadPosition(), payloadFlags());
+        }
+
+        public long getLastIndexPosition()
+        {
+            goMax(root);
+            return getCurrentIndexPos();
+        }
+
+        /**
+         * To be used only in analysis.
+         */
+        @SuppressWarnings("unused")
+        protected int payloadSize()
+        {
+            int bytes = payloadFlags();
+            return bytes > 7 ? bytes - 6 : bytes;
+        }
+    }
+
+    /**
+     * Iterator of index positions covered between two keys. Since we store prefixes only, the first and last returned
+     * values can be outside the span (and inclusiveness is not given as we cannot verify it).
+     */
+    public static class IndexPosIterator extends ValueIterator<IndexPosIterator>
+    {
+        static final long INVALID = -1;
+        long pos = INVALID;
+
+        /**
+         * @param index PartitionIndex to use for the iteration.
+         *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,

Review Comment:
   Made package-private by exploiting the protected inheritance from the test.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.
+ * <p>
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public final class TrieIndexEntry extends AbstractRowIndexEntry
+{
+    final long indexTrieRoot;
+    private final int rowIndexBlockCount;
+    private final DeletionTime deletionTime;
+
+    TrieIndexEntry(long dataFilePosition, long indexTrieRoot, int rowIndexBlockCount, DeletionTime deletionTime)
+    {
+        super(dataFilePosition);
+        this.indexTrieRoot = indexTrieRoot;
+        this.rowIndexBlockCount = rowIndexBlockCount;
+        this.deletionTime = deletionTime;
+    }
+
+    public TrieIndexEntry(long position)
+    {
+        super(position);
+        this.indexTrieRoot = -1;
+        this.rowIndexBlockCount = 0;
+        this.deletionTime = null;
+    }
+
+    @Override
+    public int blockCount()
+    {
+        return rowIndexBlockCount;
+    }
+
+    @Override
+    public SSTableFormat<?, ?> getSSTableFormat()
+    {
+        return BtiFormat.instance;
+    }
+
+    @Override
+    public void serializeForCache(DataOutputPlus out)
+    {
+        throw new AssertionError("BTI SSTables should not use key cache");
+    }
+
+    @Override
+    public DeletionTime deletionTime()
+    {
+        return deletionTime;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        throw new AssertionError("BTI SSTables index entries should not be persisted in any in-memory structure");
+    }
+
+    public void serialize(DataOutputPlus indexFile, long basePosition) throws IOException
+    {
+        assert indexTrieRoot != -1 && rowIndexBlockCount > 0 && deletionTime != null;
+        indexFile.writeUnsignedVInt(position);
+        indexFile.writeVInt(indexTrieRoot - basePosition);
+        indexFile.writeUnsignedVInt32(rowIndexBlockCount);
+        DeletionTime.serializer.serialize(deletionTime, indexFile);
+    }
+
+    /**
+     * Create an index entry. The row index trie must already have been written (by RowIndexWriter) to the row index
+     * file and its root position must be specified in trieRoot.
+     */
+    public static TrieIndexEntry create(long dataStartPosition,
+                                        long trieRoot,
+                                        DeletionTime partitionLevelDeletion,
+                                        int rowIndexBlockCount)
+    {
+        return new TrieIndexEntry(dataStartPosition, trieRoot, trieRoot == -1 ? 0 : rowIndexBlockCount, partitionLevelDeletion);
+    }
+
+    public static TrieIndexEntry deserialize(DataInputPlus in, long basePosition) throws IOException
+    {
+        long dataFilePosition = in.readUnsignedVInt();
+        long indexTrieRoot = in.readVInt() + basePosition;
+        int rowIndexCount = in.readUnsignedVInt32();

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ *
+ * The difference between this and PartitionIndexIterator is that this only uses information present in the index file

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)

Review Comment:
   For the current usage, it is equivalent, but we do permit hashless payloads in the data (see https://github.com/blambov/cassandra/blob/CASSANDRA-18398/src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md#efficiency).



##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.

Review Comment:
   Changed doc.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;

Review Comment:
   This is unreachable (we filter out branches that contain no payload in the compaction code, memtables can't construct empty paths either).
   
   It could be an assertion to alert early if e.g. a compaction rewrite fails to offer that guarantee, but
   * the case has clear handling
   * I believe we flag this case out elsewhere
   
   



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Check for a prefix and find closest smaller branch.
+            Long indexPos = prefixAndNeighbours(key, Reader::getSpecificIndexPos);
+
+            if (indexPos != null && indexPos != NOT_FOUND)
+            {
+                ResultType res = acceptor.accept(indexPos, false, key);
+                if (res != null)
+                    return res;
+            }
+
+            // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+            // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+            // prefix match that is closer than max(lesserBranch).
+            if (lesserBranch == NONE)
+                return null;
+            goMax(lesserBranch);
+            indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;

Review Comment:
   Unreachable as above.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;

Review Comment:
   I don't know what it was used for. Removed.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }
+
+        return PartitionIterator.create(partitionIndex,
+                                        metadata().partitioner,
+                                        rowIndexFile,
+                                        dfile,
+                                        left, inclusiveLeft ? -1 : 0,
+                                        right, inclusiveRight ? 0 : -1);
+    }
+
+    @Override
+    public PartitionIterator keyReader() throws IOException
+    {
+        return PartitionIterator.create(partitionIndex, metadata().partitioner, rowIndexFile, dfile);
+    }
+
+    @Override
+    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
+    {
+        // BTI does not support key sampling as it would involve walking the index or data file.
+        // Validator has an alternate solution for empty key sample lists.
+        return Collections.emptyList();
+    }
+
+    @Override
+    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
+    {
+        // Estimate the number of partitions by calculating the bytes of the sstable that are covered by the specified
+        // ranges and using the mean partition size to obtain a number of partitions from that.
+        long selectedDataSize = 0;
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            PartitionPosition left = range.left.minKeyBound();
+            if (left.compareTo(first) <= 0)
+                left = null;
+            else if (left.compareTo(last) > 0)
+                continue;   // no intersection
+
+            PartitionPosition right = range.right.minKeyBound();
+            if (range.right.isMinimum() || right.compareTo(last) >= 0)
+                right = null;
+            else if (right.compareTo(first) < 0)
+                continue;   // no intersection
+
+            if (left == null && right == null)
+                return partitionIndex.size();   // sstable is fully covered, return full partition count to avoid rounding errors
+
+            if (left == null && filterFirst())
+                left = first;
+            if (right == null && filterLast())
+                right = last;
+
+            long startPos = left != null ? getPosition(left, GE) : 0;
+            long endPos = right != null ? getPosition(right, GE) : uncompressedLength();
+            selectedDataSize += endPos - startPos;
+        }
+        return Math.round(selectedDataSize / sstableMetadata.estimatedPartitionSize.rawMean());
+    }
+
+
+    @Override
+    public UnfilteredRowIterator rowIterator(DecoratedKey key,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed,
+                                             SSTableReadsListener listener)
+    {
+        return rowIterator(null, key, getExactPosition(key, listener, true), slices, selectedColumns, reversed);
+    }
+
+    public UnfilteredRowIterator rowIterator(FileDataInput dataFileInput,
+                                             DecoratedKey key,
+                                             TrieIndexEntry indexEntry,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed)
+    {
+        if (indexEntry == null)
+            return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
+
+        if (reversed)
+            return new SSTableReversedIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+        else
+            return new SSTableIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+    }
+
+    @Override
+    public ISSTableScanner getScanner()
+    {
+        return BtiTableScanner.getScanner(this);
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Collection<Range<Token>> ranges)
+    {
+        if (ranges != null)
+            return BtiTableScanner.getScanner(this, ranges);
+        else
+            return getScanner();
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return BtiTableScanner.getScanner(this, rangeIterator);
+    }
+
+    @VisibleForTesting
+    @Override
+    public BtiTableReader cloneAndReplace(IFilter filter)
+    {
+        return unbuildTo(new Builder(descriptor).setFilter(filter), true).build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public BtiTableReader cloneWithRestoredStart(DecoratedKey restoredStart)
+    {
+        return runWithLock(ignored -> cloneAndReplace(restoredStart, OpenReason.NORMAL));
+    }
+
+    @Override
+    public BtiTableReader cloneWithNewStart(DecoratedKey newStart)
+    {
+        return runWithLock(d -> {
+            assert openReason != OpenReason.EARLY;
+            // TODO: merge with caller's firstKeyBeyond() work,to save time
+            if (newStart.compareTo(first) > 0)
+            {
+                final long dataStart = getPosition(newStart, Operator.EQ);
+                runOnClose(() -> dfile.dropPageCache(dataStart));
+            }
+
+            return cloneAndReplace(newStart, OpenReason.MOVED_START);
+        });
+    }
+
+    /**
+     * Clone this reader with the provided start and open reason, and set the clone as replacement.
+     *
+     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
+     *                 opening of compaction results).
+     * @param reason   the {@code OpenReason} for the replacement.
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    private BtiTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
+    {
+        return unbuildTo(new Builder(descriptor), true)
+               .setFirst(newFirst)
+               .setOpenReason(reason)
+               .build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public DecoratedKey firstKeyBeyond(PartitionPosition token)
+    {
+        try
+        {
+            TrieIndexEntry pos = getRowIndexEntry(token, Operator.GT, true, false, SSTableReadsListener.NOOP_LISTENER);

Review Comment:
   Yes, this is correct.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>
+    {
+        protected Reader(PartitionIndex index)
+        {
+            super(index.instantiateRebufferer(), index.root);
+        }
+
+        /**
+         * Finds a candidate for an exact key search. Returns an ifile (if positive) or dfile (if negative, using ~)
+         * position. The position returned has a low chance of being a different entry, but only if the sought key
+         * is not present in the file.
+         */
+        public long exactCandidate(DecoratedKey key)
+        {
+            // A hit must be a prefix of the byte-comparable representation of the key.
+            int b = follow(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (b != ByteSource.END_OF_STREAM && hasChildren())
+                return NOT_FOUND;
+            if (!checkHashBits(key.filterHashLowerBits()))
+                return NOT_FOUND;
+            return getCurrentIndexPos();
+        }
+
+        final boolean checkHashBits(short hashBits)
+        {
+            int bytes = payloadFlags();
+            if (bytes <= 7)
+                return bytes > 0;
+            return (buf.get(payloadPosition()) == (byte) hashBits);
+        }
+
+        public <ResultType> ResultType ceiling(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException
+        {
+            // Look for a prefix of the key. If there is one, the key it stands for could be less, equal, or greater
+            // than the required value so try that first.
+            int b = followWithGreater(key);
+            // If the prefix ended in a node with children it is only acceptable if it is a full match.
+            if (!hasChildren() || b == ByteSource.END_OF_STREAM)
+            {
+                long indexPos = getCurrentIndexPos();
+                if (indexPos != NOT_FOUND)
+                {
+                    ResultType res = acceptor.accept(indexPos, false, key);
+                    if (res != null)
+                        return res;
+                }
+            }
+            // If that was not found, the closest greater value can be used instead, and we know that
+            // it stands for a key greater than the argument.
+            if (greaterBranch == NONE)
+                return null;
+            goMin(greaterBranch);
+            long indexPos = getCurrentIndexPos();
+            if (indexPos == NOT_FOUND)
+                return null;
+
+            return acceptor.accept(indexPos, true, key);
+        }
+
+
+        public <ResultType> ResultType floor(PartitionPosition key, Acceptor<PartitionPosition, ResultType> acceptor) throws IOException

Review Comment:
   I don't remember is SAI needed it; the code is non-trivial and I'd rather not have to reimplement it if the need for it arises.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()
+    {
+        return new BtiScanningIterator();
+    }
+
+    protected class BtiScanningIterator extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>.BaseKeyScanningIterator implements Closeable
+    {
+        private PartitionIterator iterator;
+
+        protected boolean prepareToIterateRow() throws IOException
+        {
+            while (true)
+            {
+                if (startScan != -1)
+                    bytesScanned += getCurrentPosition() - startScan;
+
+                if (iterator != null)
+                {
+                    currentEntry = iterator.entry();
+                    currentKey = iterator.decoratedKey();
+                    if (currentEntry != null)
+                    {
+                        iterator.advance();
+                        return true;
+                    }
+                    iterator.close();
+                    iterator = null;
+                }
+
+                // try next range
+                if (!rangeIterator.hasNext())
+                    return false;
+                iterator = sstable.coveredKeysIterator(rangeIterator.next());
+            }
+        }
+
+        protected UnfilteredRowIterator getRowIterator(TrieIndexEntry indexEntry, DecoratedKey key)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight)
+    {
+        super(partitionIndex, left, right);
+        this.partitionIndex = partitionIndex;
+        this.partitioner = partitioner;
+        this.limit = right;
+        this.exclusiveLimit = exclusiveRight;
+        this.rowIndexFile = rowIndexFile;
+        this.dataFile = dataFile;
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex)
+    {
+        super(partitionIndex, partitionIndex.firstKey(), partitionIndex.firstKey());
+        this.partitionIndex = partitionIndex;
+        this.partitioner = null;
+        this.limit = partitionIndex.firstKey();
+        this.exclusiveLimit = -1;
+        this.rowIndexFile = null;
+        this.dataFile = null;
+
+        this.currentEntry = null;
+        this.currentKey = null;
+        this.nextEntry = null;
+        this.nextKey = null;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accum = null;
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(partitionIndex, dataFile, rowIndexFile));
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(dataInput, indexInput));
+        accum = Throwables.perform(accum, super::close);
+        Throwables.maybeFail(accum);
+    }
+
+    public DecoratedKey decoratedKey()
+    {
+        return currentKey;
+    }
+
+    public ByteBuffer key()
+    {
+        return currentKey.getKey();
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return currentEntry != null ? currentEntry.position : -1;
+    }
+
+    @Override
+    public long keyPositionForSecondaryIndex()
+    {
+        return dataPosition();
+    }
+
+    public TrieIndexEntry entry()
+    {
+        return currentEntry;
+    }
+
+    @Override
+    public boolean advance() throws IOException
+    {
+        currentKey = nextKey;
+        currentEntry = nextEntry;
+        if (currentKey != null)
+        {
+            readNext();
+            // if nextKey is null, then currentKey is the last key to be published, therefore check against any limit
+            // and suppress the partition if it is beyond the limit
+            if (nextKey == null && limit != null && currentKey.compareTo(limit) > exclusiveLimit)
+            {   // exclude last partition outside range
+                currentKey = null;
+                currentEntry = null;
+                return false;
+            }
+            return true;
+        }
+        return false;
+    }
+
+    private void readNext() throws IOException
+    {
+        long pos = nextIndexPos();
+        if (pos != PartitionIndex.NOT_FOUND)
+        {
+            if (pos >= 0)
+            {
+                seekIndexInput(pos);
+                nextKey = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(indexInput));
+                nextEntry = TrieIndexEntry.deserialize(indexInput, indexInput.getFilePointer());
+            }
+            else
+            {
+                pos = ~pos;
+                seekDataInput(pos);
+                nextKey = partitioner.decorateKey(ByteBufferUtil.readWithShortLength(dataInput));
+                nextEntry = new TrieIndexEntry(pos);
+            }
+        }
+        else
+        {
+            nextKey = null;
+            nextEntry = null;
+        }
+    }
+
+    private void seekIndexInput(long pos) throws IOException
+    {
+        if (indexInput == null)
+            indexInput = rowIndexFile.createReader(pos);
+        else
+            indexInput.seek(pos);
+    }
+
+    private void seekDataInput(long pos) throws IOException
+    {
+        if (dataInput == null)
+            dataInput = dataFile.createReader(pos);
+        else
+            dataInput.seek(pos);
+    }
+
+    @Override
+    public boolean isExhausted()
+    {
+        return currentKey == null;
+    }
+
+    @Override
+    public void reset()
+    {
+        go(root);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("TrieIndex-PartitionIndexIterator(%s)", partitionIndex.getFileHandle().path());

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);

Review Comment:
   `SSTableReaderWithFilter`'s ones override them and do use it. Adding a version without these parameters in the base class would only confuse people reading or writing this code.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            checkArgument(!online || builder.getSerializationHeader() != null);
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings({ "resource", "RedundantSuppression" })
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+                    FilterComponent.save(filter, descriptor, false);
+                }
+            }
+
+            if (builder.getFilter() == null)
+                builder.setFilter(FilterFactory.AlwaysPresent);
+
+            if (builder.getComponents().contains(Components.ROW_INDEX))
+                builder.setRowIndexFile(rowIndexFileBuilder().complete());
+
+            if (descriptor.version.hasKeyRange() && builder.getStatsMetadata() != null)
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX))
+            {
+                builder.setPartitionIndex(openPartitionIndex(builder.getFilter().isInformative()));
+                if (builder.getFirst() == null || builder.getLast() == null)
+                {
+                    builder.setFirst(builder.getPartitionIndex().firstKey());
+                    builder.setLast(builder.getPartitionIndex().lastKey());
+                }
+            }
+
+            try (CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components))
+            {
+                builder.setDataFile(dataFileBuilder(builder.getStatsMetadata()).withCompressionMetadata(compressionMetadata).complete());
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            // in case of failure, close only those components which have been opened in this try-catch block
+            Throwables.closeAndAddSuppressed(ex, builder.getPartitionIndex(), builder.getRowIndexFile(), builder.getDataFile(), builder.getFilter());
+            throw ex;
+        }
+    }
+
+    private IFilter buildBloomFilter(StatsMetadata statsMetadata) throws IOException
+    {
+        IFilter bf = null;
+
+        try (KeyReader keyReader = createKeyReader(statsMetadata))
+        {
+            bf = FilterFactory.getFilter(statsMetadata.totalRows, tableMetadataRef.getLocal().params.bloomFilterFpChance);
+
+            while (!keyReader.isExhausted())
+            {
+                DecoratedKey key = tableMetadataRef.getLocal().partitioner.decorateKey(keyReader.key());
+                bf.add(key);
+
+                keyReader.advance();
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            Throwables.closeAndAddSuppressed(ex, bf);
+            throw ex;
+        }
+
+        return bf;
+    }
+
+    private IFilter loadFilter(ValidationMetadata validationMetadata)
+    {
+        return FilterComponent.maybeLoadBloomFilter(descriptor,
+                                                    components,
+                                                    tableMetadataRef.get(),
+                                                    validationMetadata);
+    }
+
+    private PartitionIndex openPartitionIndex(boolean preload) throws IOException
+    {
+        try (FileHandle indexFile = partitionIndexFileBuilder().complete())
+        {
+            return PartitionIndex.load(indexFile, tableMetadataRef.getLocal().partitioner, preload);
+        }
+        catch (IOException ex)
+        {
+            logger.debug("Partition index file is corrupted: " + descriptor.fileFor(Components.PARTITION_INDEX), ex);
+            throw ex;
+        }
+    }
+
+    private FileHandle.Builder dataFileBuilder(StatsMetadata statsMetadata)

Review Comment:
   It does not belong in the generic loader. Introduced an intermediate type for the shared state and functionality.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            checkArgument(!online || builder.getSerializationHeader() != null);
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings({ "resource", "RedundantSuppression" })
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }
+
+        return PartitionIterator.create(partitionIndex,
+                                        metadata().partitioner,
+                                        rowIndexFile,
+                                        dfile,
+                                        left, inclusiveLeft ? -1 : 0,
+                                        right, inclusiveRight ? 0 : -1);
+    }
+
+    @Override
+    public PartitionIterator keyReader() throws IOException
+    {
+        return PartitionIterator.create(partitionIndex, metadata().partitioner, rowIndexFile, dfile);
+    }
+
+    @Override
+    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
+    {
+        // BTI does not support key sampling as it would involve walking the index or data file.
+        // Validator has an alternate solution for empty key sample lists.
+        return Collections.emptyList();
+    }
+
+    @Override
+    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
+    {
+        // Estimate the number of partitions by calculating the bytes of the sstable that are covered by the specified
+        // ranges and using the mean partition size to obtain a number of partitions from that.
+        long selectedDataSize = 0;
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            PartitionPosition left = range.left.minKeyBound();
+            if (left.compareTo(first) <= 0)
+                left = null;
+            else if (left.compareTo(last) > 0)
+                continue;   // no intersection
+
+            PartitionPosition right = range.right.minKeyBound();
+            if (range.right.isMinimum() || right.compareTo(last) >= 0)
+                right = null;
+            else if (right.compareTo(first) < 0)
+                continue;   // no intersection
+
+            if (left == null && right == null)
+                return partitionIndex.size();   // sstable is fully covered, return full partition count to avoid rounding errors
+
+            if (left == null && filterFirst())
+                left = first;
+            if (right == null && filterLast())
+                right = last;
+
+            long startPos = left != null ? getPosition(left, GE) : 0;
+            long endPos = right != null ? getPosition(right, GE) : uncompressedLength();
+            selectedDataSize += endPos - startPos;
+        }
+        return Math.round(selectedDataSize / sstableMetadata.estimatedPartitionSize.rawMean());
+    }
+
+
+    @Override
+    public UnfilteredRowIterator rowIterator(DecoratedKey key,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed,
+                                             SSTableReadsListener listener)
+    {
+        return rowIterator(null, key, getExactPosition(key, listener, true), slices, selectedColumns, reversed);
+    }
+
+    public UnfilteredRowIterator rowIterator(FileDataInput dataFileInput,
+                                             DecoratedKey key,
+                                             TrieIndexEntry indexEntry,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed)
+    {
+        if (indexEntry == null)
+            return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
+
+        if (reversed)
+            return new SSTableReversedIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+        else
+            return new SSTableIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+    }
+
+    @Override
+    public ISSTableScanner getScanner()
+    {
+        return BtiTableScanner.getScanner(this);
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Collection<Range<Token>> ranges)
+    {
+        if (ranges != null)
+            return BtiTableScanner.getScanner(this, ranges);
+        else
+            return getScanner();
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return BtiTableScanner.getScanner(this, rangeIterator);
+    }
+
+    @VisibleForTesting
+    @Override
+    public BtiTableReader cloneAndReplace(IFilter filter)
+    {
+        return unbuildTo(new Builder(descriptor).setFilter(filter), true).build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public BtiTableReader cloneWithRestoredStart(DecoratedKey restoredStart)
+    {
+        return runWithLock(ignored -> cloneAndReplace(restoredStart, OpenReason.NORMAL));
+    }
+
+    @Override
+    public BtiTableReader cloneWithNewStart(DecoratedKey newStart)
+    {
+        return runWithLock(d -> {
+            assert openReason != OpenReason.EARLY;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()
+    {
+        return new BtiScanningIterator();
+    }
+
+    protected class BtiScanningIterator extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>.BaseKeyScanningIterator implements Closeable
+    {
+        private PartitionIterator iterator;
+
+        protected boolean prepareToIterateRow() throws IOException

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164686136


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        LongStack rowOffsets = new LongStack();

Review Comment:
   ```suggestion
           final LongStack rowOffsets = new LongStack();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167088675


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;

Review Comment:
   nit: short message for the `assert`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167076380


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings("resource")

Review Comment:
   ```suggestion
       @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167254415


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         *
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1195436615


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);

Review Comment:
   Scratch that, this is a path I actually gave up on covering (path is only engaged with early open).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1195176972


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173522269


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public boolean hasIsTransient()
+        {
+            return version.compareTo("ca") >= 0;
+        }
+
+        @Override
+        public boolean hasMetadataChecksum()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasOldBfFormat()
+        {
+            return hasOldBfFormat;
+        }
+
+        @Override
+        public boolean hasAccurateMinMax()
+        {
+            return hasAccurateMinMax;
+        }
+
+        public boolean hasLegacyMinMax()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean hasOriginatingHostId()
+        {
+            return hasOriginatingHostId;
+        }
+
+        @Override
+        public boolean hasImprovedMinMax() {
+            return hasImprovedMinMax;
+        }
+
+        @Override
+        public boolean hasPartitionLevelDeletionsPresenceMarker()
+        {
+            return hasPartitionLevelDeletionsPresenceMarker;
+        }
+
+        @Override
+        public boolean hasKeyRange()
+        {
+            return hasKeyRange;
+        }
+
+        @Override
+        public boolean isCompatible()
+        {
+            return version.compareTo(earliest_supported_version) >= 0 && version.charAt(0) <= current_version.charAt(0);
+        }
+
+        // this field is not present in DSE

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171656504


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException
+     */
+    public static boolean equalsWithShortLength(FileDataInput in, ByteBuffer toMatch) throws IOException

Review Comment:
   I think I just meant the whole method. I looked at a couple places, like `QueryWithIndexedSSTableTest` and `SSTableReaderTest`, but I'm probably missing something obvious.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1169272356


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).

Review Comment:
   Although all the links to external bookstores appear to be broken :D



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179680680


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)

Review Comment:
   ```suggestion
       protected IndexInfo readPayload(int ppos, int flags)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179704381


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)

Review Comment:
   nit: Might be a good idea to pull up a named constant for `-1` in `Walker`, use it there and here. Not super particular about what it's called, as long as it connotes something about the branch not existing...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179712576


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")

Review Comment:
   ```suggestion
       @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179726581


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ *     - data file position if the partition is small enough to not need an index
+ *     - row index file position if the partition has a row index
+ * plus
+ *     - the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ *
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ *
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ *
+ * To read the index one must obtain a thread-unsafe Reader or IndexPosIterator.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>

Review Comment:
   nit: `@NotThreadSafe`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179754275


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    protected TransactionalProxy txnProxy()

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1170004427


##########
src/java/org/apache/cassandra/db/DeletionTime.java:
##########
@@ -191,4 +201,4 @@ public long serializedSize(DeletionTime delTime)
                  + TypeSizes.sizeof(delTime.markedForDeleteAt());
         }
     }
-}
+}

Review Comment:
   Reverted



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (IOException e)
+        {
+            outputHandler.warn("Index is unreadable.");

Review Comment:
   Added stack, changed handled to catch all errors, and added "scrubbing will continue without index."



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries

Review Comment:
   This is a comment for the next several methods that implement page-management functionality.
   
   Removed now, together with the methods which are currently unused.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();

Review Comment:
   Moved most functionality into the shared `SortedTableVerifier`.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())
+                    throw new CompactionInterruptedException(verifyInfo.getCompactionInfo());
+
+                rowStart = dataFile.getFilePointer();
+                outputHandler.debug("Reading row at %d", rowStart);
+
+                DecoratedKey key = null;
+                try
+                {
+                    key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
+                }
+                catch (Throwable th)
+                {
+                    throwIfFatal(th);
+                    // check for null key below
+                }
+
+                if (options.checkOwnsTokens && ownedRanges.size() > 0 && !(cfs.getPartitioner() instanceof LocalPartitioner))
+                {
+                    try
+                    {
+                        rangeOwnHelper.validate(key);
+                    }
+                    catch (Throwable t)
+                    {
+                        outputHandler.warn(t, "Key %s in sstable %s not owned by local ranges %s", key, sstable, ownedRanges);
+                        markAndThrow(t);
+                    }
+                }
+
+                ByteBuffer currentIndexKey = indexIterator.key();
+                long nextRowPositionFromIndex = 0;
+                try
+                {
+                    nextRowPositionFromIndex = indexIterator.advance()
+                                               ? indexIterator.dataPosition()
+                                               : dataFile.length();
+                }
+                catch (Throwable th)
+                {
+                    markAndThrow(th);
+                }
+
+                long dataStart = dataFile.getFilePointer();
+                long dataStartFromIndex = currentIndexKey == null
+                                          ? -1
+                                          : rowStart + 2 + currentIndexKey.remaining();
+
+                long dataSize = nextRowPositionFromIndex - dataStartFromIndex;
+                // avoid an NPE if key is null
+                String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey());
+                outputHandler.debug("row %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSize));
+
+                try
+                {
+                    if (key == null || dataSize > dataFile.length())
+                        markAndThrow(new RuntimeException(String.format("key = %s, dataSize=%d, dataFile.length() = %d", key, dataSize, dataFile.length())));
+
+                    //mimic the scrub read path, intentionally unused
+                    try (UnfilteredRowIterator ignored = SSTableIdentityIterator.create(sstable, dataFile, key)) {
+                        // no-op, just open and close
+                    }
+
+                    if ((prevKey != null && prevKey.compareTo(key) > 0) || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex)
+                        markAndThrow(new RuntimeException("Key out of order: previous = " + prevKey + " : current = " + key));
+
+                    goodRows++;
+                    prevKey = key;
+
+
+                    outputHandler.debug("Row %s at %s valid, moving to next row at %s ", goodRows, rowStart, nextRowPositionFromIndex);
+                    dataFile.seek(nextRowPositionFromIndex);
+                }
+                catch (Throwable th)
+                {
+                    markAndThrow(th);
+                }
+            }
+        }
+        catch (Throwable t)
+        {
+            throw Throwables.propagate(t);

Review Comment:
   Fixed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableIterator.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ *  Unfiltered row iterator over a BTI SSTable.
+ */
+class SSTableIterator extends AbstractSSTableIterator<AbstractRowIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableIterator(BtiTableReader sstable,
+                           FileDataInput file,
+                           DecoratedKey key,
+                           AbstractRowIndexEntry indexEntry,
+                           Slices slices,
+                           ColumnFilter columns,
+                           FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(AbstractRowIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ForwardIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ForwardReader(file, shouldCloseFile);
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return next;
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    public boolean isReverseOrder()
+    {
+        return false;
+    }
+
+    private class ForwardIndexedReader extends ForwardReader
+    {
+        private final RowIndexReader indexReader;
+        long basePosition;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        LongStack rowOffsets = new LongStack();

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        Unfiltered next = null;
+        boolean foundLessThan;
+        long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();
+            currentPosition = file.getFilePointer();
+            foundLessThan = false;
+            // This is a copy of handlePreSliceData which also checks currentPosition < stopPosition.
+            // Not extracted to method as we need both marker and currentPosition.
+            if (filterStart)
+            {
+                while (currentPosition < stopPosition && deserializer.hasNext() && deserializer.compareNextTo(start) <= 0)
+                {
+                    if (deserializer.nextIsRow())
+                        deserializer.skipNext();
+                    else
+                        updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                    currentPosition = file.getFilePointer();
+                    foundLessThan = true;
+                }
+            }
+
+            // We've reached the beginning of our queried slice. If we have an open marker
+            // we should return that at the end of the slice to close the deletion.
+            if (openMarker != null)
+                blockOpenMarker = new RangeTombstoneBoundMarker(start, openMarker);
+
+
+            // Now deserialize everything until we reach our requested end (if we have one)
+            // See SSTableIterator.ForwardRead.computeNext() for why this is a strict inequality below: this is the same
+            // reasoning here.
+            while (currentPosition < stopPosition && deserializer.hasNext()
+                   && (!filterEnd || deserializer.compareNextTo(slice.end()) < 0))
+            {
+                rowOffsets.push(currentPosition);
+                if (deserializer.nextIsRow())
+                    deserializer.skipNext();
+                else
+                    updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                currentPosition = file.getFilePointer();
+            }
+
+            // If we have an open marker, we should output that first, unless end is not being filtered
+            // (i.e. it's either top (where a marker can't be open) or we placed that marker during previous block).
+            if (openMarker != null && filterEnd)
+            {
+                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                blockCloseMarker = new RangeTombstoneBoundMarker(slice.end(), openMarker);
+                openMarker = null;
+            }
+        }
+    }
+
+    private class ReverseIndexedReader extends ReverseReader
+    {
+        private RowIndexReverseIterator indexReader;
+        final TrieIndexEntry indexEntry;
+        long basePosition;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings("resource")
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight) throws IOException

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+
+                    if (online)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;

Review Comment:
   Removed 118



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).

Review Comment:
   Changed to the .com site. If you have a better suggestion on a neutral site for the reference, I'd be happy to change it.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (IOException e)
+        {
+            outputHandler.warn("Index is unreadable.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())
+                    cfs.metadata.getLocal().partitionKeyType.validate(raw);
+                key = sstable.decorateKey(raw);
+            }
+            catch (Throwable th)
+            {
+                keyReadError = th;
+                throwIfFatal(th);
+                // check for null key below
+            }
+
+            // position of the partition in a data file, it points to the beginning of the partition key
+            long dataStartFromIndex = -1;
+            // size of the partition (including partition key)
+            long dataSizeFromIndex = -1;
+            ByteBuffer currentIndexKey = null;
+            if (indexAvailable())
+            {
+                currentIndexKey = indexIterator.key();
+                dataStartFromIndex = indexIterator.dataPosition();
+                if (!indexIterator.isExhausted())
+                {
+                    try
+                    {
+                        indexIterator.advance();
+                        if (!indexIterator.isExhausted())
+                            dataSizeFromIndex = indexIterator.dataPosition() - dataStartFromIndex;
+                    }
+                    catch (Throwable th)
+                    {
+                        throwIfFatal(th);
+                        outputHandler.warn(th, "Failed to advance to the next index position. Index is corrupted. " +

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())

Review Comment:
   Moved most functionality into the shared `SortedTableVerifier`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1173522053


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)
+    {
+        SortedTableScrubber.deleteOrphanedComponents(descriptor, components);
+    }
+
+    private void delete(Descriptor desc, List<Component> components)
+    {
+        logger.info("Deleting sstable: {}", desc);
+
+        if (components.remove(SSTableFormat.Components.DATA))
+            components.add(0, SSTableFormat.Components.DATA); // DATA component should be first
+
+        for (Component component : components)
+        {
+            logger.trace("Deleting component {} of {}", component, desc);
+            desc.fileFor(component).deleteIfExists();
+        }
+    }
+
+    @Override
+    public void delete(Descriptor desc)
+    {
+        try
+        {
+            delete(desc, Lists.newArrayList(Sets.intersection(allComponents(), desc.discoverComponents())));
+        }
+        catch (Throwable t)
+        {
+            JVMStabilityInspector.inspectThrowable(t);
+        }
+    }
+
+    static class BtiTableReaderFactory implements SSTableReaderFactory<BtiTableReader, BtiTableReader.Builder>
+    {
+        @Override
+        public SSTableReader.Builder<BtiTableReader, BtiTableReader.Builder> builder(Descriptor descriptor)
+        {
+            return new BtiTableReader.Builder(descriptor);
+        }
+
+        @Override
+        public SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder> loadingBuilder(Descriptor descriptor, TableMetadataRef tableMetadataRef, Set<Component> components)
+        {
+            return new BtiTableReaderLoadingBuilder(new SSTable.Builder<>(descriptor).setTableMetadataRef(tableMetadataRef)
+                                                                                     .setComponents(components));
+        }
+
+        @Override
+        public Pair<DecoratedKey, DecoratedKey> readKeyRange(Descriptor descriptor, IPartitioner partitioner) throws IOException
+        {
+            return PartitionIndex.readFirstAndLastKey(descriptor.fileFor(Components.PARTITION_INDEX), partitioner);
+        }
+
+        @Override
+        public Class<BtiTableReader> getReaderClass()
+        {
+            return BtiTableReader.class;
+        }
+    }
+
+    static class BtiTableWriterFactory implements SSTableWriterFactory<BtiTableWriter, BtiTableWriter.Builder>
+    {
+        @Override
+        public BtiTableWriter.Builder builder(Descriptor descriptor)
+        {
+            return new BtiTableWriter.Builder(descriptor);
+        }
+
+        @Override
+        public long estimateSize(SSTableWriter.SSTableSizeParameters parameters)
+        {
+            return (long) ((parameters.partitionCount() // index entries
+                            + parameters.partitionCount() // keys in data file
+                            + parameters.dataSize()) // data
+                           * 1.2); // bloom filter and row index overhead
+        }
+    }
+
+    // versions are denoted as [major][minor].  Minor versions must be forward-compatible:
+    // new fields are allowed in e.g. the metadata component, but fields can't be removed
+    // or have their size changed.
+    //
+    static class BtiVersion extends Version
+    {
+        public static final String current_version = "da";
+        public static final String earliest_supported_version = "ca";
+
+        // aa (DSE 6.0): trie index format
+        // ab (DSE pre-6.8): ILLEGAL - handled as 'b' (predates 'ba'). Pre-GA "LABS" releases of DSE 6.8 used this
+        //                   sstable version.
+        // ac (DSE 6.0.11, 6.7.6): corrected sstable min/max clustering (DB-3691/CASSANDRA-14861)
+        // ad (DSE 6.0.14, 6.7.11): added hostId of the node from which the sstable originated (DB-4629)
+        // b  (DSE early 6.8 "LABS") has some of 6.8 features but not all
+        // ba (DSE 6.8): encrypted indices and metadata
+        //               new BloomFilter serialization format
+        //               add incremental NodeSync information to metadata
+        //               improved min/max clustering representation
+        //               presence marker for partition level deletions
+        // bb (DSE 6.8.5): added hostId of the node from which the sstable originated (DB-4629)
+        // versions aa-bz are not supported in OSS
+        // ca (DSE-DB aka Stargazer based on OSS 4.0): all OSS fields + all DSE fields in DSE serialization format
+        // da - same as ca but in OSS serialization format
+        // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
+
+        private final boolean isLatestVersion;
+
+        /**
+         * DB-2648/CASSANDRA-9067: DSE 6.8/OSS 4.0 bloom filter representation changed (bitset data is no longer stored
+         * as BIG_ENDIAN longs, which avoids some redundant bit twiddling).
+         */
+        private final int correspondingMessagingVersion;
+
+        private final boolean hasOldBfFormat;
+        private final boolean hasAccurateMinMax;
+        private final boolean hasImprovedMinMax;
+        private final boolean hasKeyRange;
+        private final boolean hasPartitionLevelDeletionsPresenceMarker;
+        private final boolean hasOriginatingHostId;
+
+        BtiVersion(String version)
+        {
+            super(instance, version = mapAb(version));
+
+            isLatestVersion = version.compareTo(current_version) == 0;
+            hasOldBfFormat = version.compareTo("b") < 0;
+            hasAccurateMinMax = version.compareTo("ac") >= 0;
+            hasOriginatingHostId = version.matches("(a[d-z])|(b[b-z])") || version.compareTo("ca") >= 0;
+            hasImprovedMinMax = version.compareTo("ba") >= 0;
+            hasKeyRange = version.compareTo("da") >= 0;
+            hasPartitionLevelDeletionsPresenceMarker = version.compareTo("ba") >= 0;
+            correspondingMessagingVersion = MessagingService.VERSION_40;
+        }
+
+        // this is for the ab version which was used in the LABS, and then has been renamed to ba
+        private static String mapAb(String version)
+        {
+            return "ab".equals(version) ? "ba" : version;
+        }
+
+        @Override
+        public boolean isLatestVersion()
+        {
+            return isLatestVersion;
+        }
+
+        // this field is not present in DSE
+        @Override
+        public int correspondingMessagingVersion()
+        {
+            return correspondingMessagingVersion;
+        }
+
+        @Override
+        public boolean hasCommitLogLowerBound()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasMaxCompressedLength()
+        {
+            return true;
+        }
+
+        @Override
+        public boolean hasPendingRepair()
+        {
+            return true;
+        }
+
+        // this field is not present in DSE

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162025086


##########
conf/cassandra.yaml:
##########
@@ -1924,8 +1924,14 @@ drop_compact_storage_enabled: false
 # which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
 # which will be used to recognize the format type - in particular that name will be used in sstable file names and in
 # stream headers so the name has to be the same for the same format across all the nodes in the cluster.
+# The first entry in this list is the format that will be used for newly-created SSTables. The other formats given
+# will be used to read any SSTables present in the data directories or streamed.
 sstable_formats:
   - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
     parameters:
       id: 0
       name: big
+  - class_name: org.apache.cassandra.io.sstable.format.bti.BtiFormat
+    parameters:
+      id: 1
+      name: bti

Review Comment:
   I wasn't able to look at the original SSTable format API configuration work, but there are a few things that worry me a bit:
   
   1.) Hard to know without actually writing the code, but it feels like some things would have been a bit simpler if we avoided `ParameterizedClass` and just used a `EncryptionOptions`-eqsue configuration object to contain the structure and validation logic for the formats.
   
   2.) There are a couple things here that feel very dangerous to expose to an operator via local configuration, with both local and cluster-wide implications.
   
   The first is having the `id` concept specified in config. If we start a node, write some SSTables, bring the node down, change the id of the primary format, then start the node again, I think we can break things like `KeyCacheSerializer#deserialize()`, which have expectations around previously written `id`s and how they map to formats. The ID concept feels like it should be statically and globally defined by the format itself. If you create a custom format, it should simply avoid conflicting w/ built-in `id`s/`name`s. (This may become a distributed problem around streaming too through a local configuration change on another node?)
   
   Second, while specifying a primary format (to write new SSTables) is necessary, allowing that to be determined by physical order in the YAML should IMO be avoided. It isn't a catastrophic risk, but one where you could silently write the wrong format for an accidental ordering mistake. The selection of a primary format should be explicit/not be a mystery in the absence of inline comments.
   
   Last (and I feel like we had this discussion around the Memtable API a while back), having to specify anything about the **built-in** formats in YAML space feels unnecessary. We'd still need something like this to specify custom formats, etc.
   
   I'm not pushing for a [Lucene-style SPI approach](https://www.elastic.co/blog/what-is-an-apache-lucene-codec) or anything in particular, but we need to at least discuss the above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164683730


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())
+                    throw new CompactionInterruptedException(verifyInfo.getCompactionInfo());
+
+                rowStart = dataFile.getFilePointer();
+                outputHandler.debug("Reading row at %d", rowStart);
+
+                DecoratedKey key = null;
+                try
+                {
+                    key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
+                }
+                catch (Throwable th)
+                {
+                    throwIfFatal(th);
+                    // check for null key below
+                }
+
+                if (options.checkOwnsTokens && ownedRanges.size() > 0 && !(cfs.getPartitioner() instanceof LocalPartitioner))
+                {
+                    try
+                    {
+                        rangeOwnHelper.validate(key);
+                    }
+                    catch (Throwable t)
+                    {
+                        outputHandler.warn(t, "Key %s in sstable %s not owned by local ranges %s", key, sstable, ownedRanges);
+                        markAndThrow(t);
+                    }
+                }
+
+                ByteBuffer currentIndexKey = indexIterator.key();
+                long nextRowPositionFromIndex = 0;
+                try
+                {
+                    nextRowPositionFromIndex = indexIterator.advance()
+                                               ? indexIterator.dataPosition()
+                                               : dataFile.length();
+                }
+                catch (Throwable th)
+                {
+                    markAndThrow(th);
+                }
+
+                long dataStart = dataFile.getFilePointer();
+                long dataStartFromIndex = currentIndexKey == null
+                                          ? -1
+                                          : rowStart + 2 + currentIndexKey.remaining();
+
+                long dataSize = nextRowPositionFromIndex - dataStartFromIndex;
+                // avoid an NPE if key is null
+                String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey());
+                outputHandler.debug("row %s is %s", keyName, FBUtilities.prettyPrintMemory(dataSize));
+
+                try
+                {
+                    if (key == null || dataSize > dataFile.length())
+                        markAndThrow(new RuntimeException(String.format("key = %s, dataSize=%d, dataFile.length() = %d", key, dataSize, dataFile.length())));
+
+                    //mimic the scrub read path, intentionally unused
+                    try (UnfilteredRowIterator ignored = SSTableIdentityIterator.create(sstable, dataFile, key)) {
+                        // no-op, just open and close
+                    }
+
+                    if ((prevKey != null && prevKey.compareTo(key) > 0) || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex)
+                        markAndThrow(new RuntimeException("Key out of order: previous = " + prevKey + " : current = " + key));
+
+                    goodRows++;
+                    prevKey = key;
+
+
+                    outputHandler.debug("Row %s at %s valid, moving to next row at %s ", goodRows, rowStart, nextRowPositionFromIndex);
+                    dataFile.seek(nextRowPositionFromIndex);
+                }
+                catch (Throwable th)
+                {
+                    markAndThrow(th);
+                }
+            }
+        }
+        catch (Throwable t)
+        {
+            throw Throwables.propagate(t);

Review Comment:
   nit: I think we removed deprecated usages of `propagate()` in CASSANDRA-14218, FWIW



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178379175


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)
+    {
+        return (BtiTableReader) sstr;
+    }
+
+    @Override
+    public BtiTableWriter cast(SSTableWriter sstw)
+    {
+        return (BtiTableWriter) sstw;
+    }
+
+    @Override
+    public MetricsProviders getFormatSpecificMetricsProviders()
+    {
+        return BtiTableSpecificMetricsProviders.instance;
+    }
+
+    @Override
+    public void deleteOrphanedComponents(Descriptor descriptor, Set<Component> components)

Review Comment:
   Should we just pull this method up to `AbstractSSTableFormat`? (Same for `BigFormat` I think...)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179667186


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException

Review Comment:
   ```suggestion
           private void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179680680


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)

Review Comment:
   ```suggestion
       protected IndexInfo readPayload(int ppos, int flags)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178399233


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.java:
##########
@@ -0,0 +1,451 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.GaugeProvider;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.MetricsProviders;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.filter.BloomFilterMetrics;
+import org.apache.cassandra.io.sstable.format.AbstractSSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Bigtable format with trie indices
+ */
+public class BtiFormat extends AbstractSSTableFormat<BtiTableReader, BtiTableWriter>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiFormat.class);
+
+    public static final BtiFormat instance = new BtiFormat();
+
+    public static final Version latestVersion = new BtiVersion(BtiVersion.current_version);
+    static final BtiTableReaderFactory readerFactory = new BtiTableReaderFactory();
+    static final BtiTableWriterFactory writerFactory = new BtiTableWriterFactory();
+
+    public static class Components extends AbstractSSTableFormat.Components
+    {
+        public static class Types extends AbstractSSTableFormat.Components.Types
+        {
+            public static final Component.Type PARTITION_INDEX = Component.Type.createSingleton("PARTITION_INDEX", "Partitions.db", BtiFormat.class);
+            public static final Component.Type ROW_INDEX = Component.Type.createSingleton("ROW_INDEX", "Rows.db", BtiFormat.class);
+        }
+
+        public final static Component PARTITION_INDEX = Types.PARTITION_INDEX.getSingleton();
+
+        public final static Component ROW_INDEX = Types.ROW_INDEX.getSingleton();
+
+        private final static Set<Component> STREAMING_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                   PARTITION_INDEX,
+                                                                                   ROW_INDEX,
+                                                                                   STATS,
+                                                                                   COMPRESSION_INFO,
+                                                                                   FILTER,
+                                                                                   DIGEST,
+                                                                                   CRC);
+
+        private final static Set<Component> PRIMARY_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                 PARTITION_INDEX);
+
+        private final static Set<Component> MUTABLE_COMPONENTS = ImmutableSet.of(STATS);
+
+        private static final Set<Component> UPLOAD_COMPONENTS = ImmutableSet.of(DATA,
+                                                                                PARTITION_INDEX,
+                                                                                ROW_INDEX,
+                                                                                COMPRESSION_INFO,
+                                                                                STATS);
+
+        private static final Set<Component> BATCH_COMPONENTS = ImmutableSet.of(DATA,
+                                                                               PARTITION_INDEX,
+                                                                               ROW_INDEX,
+                                                                               COMPRESSION_INFO,
+                                                                               FILTER,
+                                                                               STATS);
+
+        private final static Set<Component> ALL_COMPONENTS = ImmutableSet.of(DATA,
+                                                                             PARTITION_INDEX,
+                                                                             ROW_INDEX,
+                                                                             STATS,
+                                                                             COMPRESSION_INFO,
+                                                                             FILTER,
+                                                                             DIGEST,
+                                                                             CRC,
+                                                                             TOC);
+
+        private final static Set<Component> GENERATED_ON_LOAD_COMPONENTS = ImmutableSet.of(FILTER);
+    }
+
+
+    private BtiFormat()
+    {
+
+    }
+
+    public static BtiFormat getInstance()
+    {
+        return instance;
+    }
+
+    public static boolean isDefault()
+    {
+        return getInstance().getType() == Type.current();
+    }
+
+    @Override
+    public Version getLatestVersion()
+    {
+        return latestVersion;
+    }
+
+    @Override
+    public Version getVersion(String version)
+    {
+        return new BtiVersion(version);
+    }
+
+    @Override
+    public BtiTableWriterFactory getWriterFactory()
+    {
+        return writerFactory;
+    }
+
+    @Override
+    public BtiTableReaderFactory getReaderFactory()
+    {
+        return readerFactory;
+    }
+
+    @Override
+    public Set<Component> streamingComponents()
+    {
+        return Components.STREAMING_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> primaryComponents()
+    {
+        return Components.PRIMARY_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> batchComponents()
+    {
+        return Components.BATCH_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> uploadComponents()
+    {
+        return Components.UPLOAD_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> mutableComponents()
+    {
+        return Components.MUTABLE_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> allComponents()
+    {
+        return Components.ALL_COMPONENTS;
+    }
+
+    @Override
+    public Set<Component> generatedOnLoadComponents()
+    {
+        return Components.GENERATED_ON_LOAD_COMPONENTS;
+    }
+
+    @Override
+    public SSTableFormat.KeyCacheValueSerializer<BtiTableReader, TrieIndexEntry> getKeyCacheValueSerializer()
+    {
+        throw new AssertionError("BTI sstables do not use key cache");
+    }
+
+    @Override
+    public IScrubber getScrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, OutputHandler outputHandler, IScrubber.Options options)
+    {
+        Preconditions.checkArgument(cfs.metadata().equals(transaction.onlyOne().metadata()));
+        return new BtiTableScrubber(cfs, transaction, outputHandler, options);
+    }
+
+    @Override
+    public BtiTableReader cast(SSTableReader sstr)

Review Comment:
   nit: Are these `cast()` methods unused?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178449885


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();

Review Comment:
   nit: Is this codepath tested in `SSTableWriterTest` or something similar? I haven't been able to find anything yet...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179725736


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>

Review Comment:
   I know it's not that hard to discover, but whenever we have a reader/writer pair, it's kind of nice to have `link` elements in the class JavaDoc that points from reader -> writer and writer -> reader. In this case, something like...
   
   ```
   /**
    * Writes tables that can later be read by {@link BtiTableReader} blah blah
    */
   ```
   Same w/ `RowIndexWriter/Reader` and `PartitionIndex/Builder`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179709602


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ * <p>
+ * Uses IncrementalTrieWriter to build a trie of index section separators of the shortest possible length such that

Review Comment:
   ```suggestion
    * Uses {@link IncrementalTrieWriter} to build a trie of index section separators of the shortest possible length such that
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181902147


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }

Review Comment:
   nit: Any tests where we actually enter any of the 3 conditional bodies above?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180763281


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;

Review Comment:
   ```suggestion
                   int payloadBits = 7 + size;
   ```
   nit: Just declare when it's assigned?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180758597


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException

Review Comment:
   ```suggestion
           private void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181891611


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException

Review Comment:
   ```suggestion
       private PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179491600


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181731873


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);

Review Comment:
   nit: Any tests that hit this codepath?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182998610


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");

Review Comment:
   ...or perhaps just the SSTable `descriptor toString()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183084718


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185307931


##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -219,6 +233,55 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
                 return reader;
             }
         }
+        else if (format == BtiFormat.instance)
+        {
+            Set<Component> components = ImmutableSet.of(Components.DATA, BtiFormat.Components.PARTITION_INDEX, BtiFormat.Components.ROW_INDEX, Components.FILTER, Components.TOC);
+            for (Component component : components)
+            {
+                File file = descriptor.fileFor(component);
+                file.createFileIfNotExists();
+            }
+            // .complete() with size to make sstable.onDiskLength work
+            try (FileHandle fileHandle = new FileHandle.Builder(tempFile).bufferSize(size).withLengthOverride(size).complete())
+            {
+                if (size > 0)
+                {
+                    try
+                    {
+                        File file = descriptor.fileFor(Components.DATA);
+                        Util.setFileLength(file, size);
+                    }
+                    catch (IOException e)
+                    {
+                        throw new RuntimeException(e);
+                    }
+                }

Review Comment:
   nit: This little chunk of code looks duplicated above, but not a huge deal...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185514594


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()

Review Comment:
   nit: Needs `@BeforeClass` to actually be called?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185330988


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ * <ul>
+ *   <li> calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   <li> delaying writing any part of a completed node until its branch size is above the page size
+ *   <li> laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   <li> adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ * </ul>
+ * <p>
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ * <p>
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * <pre>
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * </pre>
+ * where lines denote page boundaries.
+ * <p>
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * <p>
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * <p>
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+                child.filePos = write(child);
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+                || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    protected String dumpNode(Node<VALUE> node, long nodePosition)
+    {
+        StringBuilder res = new StringBuilder(String.format("At %,d(%x) type %s child count %s nodeSize %,d branchSize %,d %s%s%n",
+                                                            nodePosition, nodePosition,
+                                                            TrieNode.typeFor(node, nodePosition), node.childCount(), node.nodeSize, node.branchSize,
+                                                            node.hasOutOfPageChildren ? "C" : "",
+                                                            node.hasOutOfPageInBranch ? "B" : ""));
+        for (Node<VALUE> child : node.children)
+            res.append(String.format("Child %2x at %,d(%x) type %s child count %s size %s nodeSize %,d branchSize %,d %s%s%n",
+                                     child.transition & 0xFF,
+                                     child.filePos,
+                                     child.filePos,
+                                     child.children != null ? TrieNode.typeFor(child, child.filePos) : "n/a",
+                                     child.children != null ? child.childCount() : "n/a",
+                                     child.children != null ? serializer.sizeofNode(child, child.filePos) : "n/a",
+                                     child.nodeSize,
+                                     child.branchSize,
+                                     child.hasOutOfPageChildren ? "C" : "",
+                                     child.hasOutOfPageInBranch ? "B" : ""));
+
+        return res.toString();
+    }
+
+    @Override
+    public PartialTail makePartialRoot() throws IOException
+    {
+        // The expectation is that the partial tail will be in memory, so we don't bother with page-fitting.
+        // We could also send some completed children to disk, but that could make suboptimal layout choices, so we'd
+        // rather not. Just write anything not written yet to a buffer, from bottom to top, and we're done.
+        try (DataOutputBuffer buf = new DataOutputBuffer())
+        {
+            PTail tail = new PTail();
+            // Readers ask rebufferers for page-aligned positions, so make sure tail starts at one.
+            // "Padding" of the cutoff point may leave some unaddressable space in the constructed file view.
+            // Nothing will point to it, though, so that's fine.
+            tail.cutoff = dest.paddedPosition();
+            tail.count = count;
+            tail.root = writePartial(stack.getFirst(), buf, tail.cutoff);
+            tail.tail = buf.asNewBuffer();
+            return tail;
+        }
+    }
+
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException

Review Comment:
   nit: @SuppressWarnings("DuplicatedCode")



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183073134


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight)
+    {
+        super(partitionIndex, left, right);
+        this.partitionIndex = partitionIndex;
+        this.partitioner = partitioner;
+        this.limit = right;
+        this.exclusiveLimit = exclusiveRight;
+        this.rowIndexFile = rowIndexFile;
+        this.dataFile = dataFile;
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex)
+    {
+        super(partitionIndex, partitionIndex.firstKey(), partitionIndex.firstKey());
+        this.partitionIndex = partitionIndex;
+        this.partitioner = null;
+        this.limit = partitionIndex.firstKey();
+        this.exclusiveLimit = -1;
+        this.rowIndexFile = null;
+        this.dataFile = null;
+
+        this.currentEntry = null;
+        this.currentKey = null;
+        this.nextEntry = null;
+        this.nextKey = null;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accum = null;
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(partitionIndex, dataFile, rowIndexFile));
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(dataInput, indexInput));
+        accum = Throwables.perform(accum, super::close);
+        Throwables.maybeFail(accum);
+    }
+
+    public DecoratedKey decoratedKey()
+    {
+        return currentKey;
+    }
+
+    public ByteBuffer key()
+    {
+        return currentKey.getKey();
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return currentEntry != null ? currentEntry.position : -1;
+    }
+
+    @Override
+    public long keyPositionForSecondaryIndex()
+    {
+        return dataPosition();
+    }
+
+    public TrieIndexEntry entry()
+    {
+        return currentEntry;
+    }
+
+    @Override
+    public boolean advance() throws IOException
+    {
+        currentKey = nextKey;
+        currentEntry = nextEntry;
+        if (currentKey != null)
+        {
+            readNext();
+            // if nextKey is null, then currentKey is the last key to be published, therefore check against any limit
+            // and suppress the partition if it is beyond the limit
+            if (nextKey == null && limit != null && currentKey.compareTo(limit) > exclusiveLimit)
+            {   // exclude last partition outside range
+                currentKey = null;

Review Comment:
   nit: Any test coverage for this path?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185248705


##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableVerifier.java:
##########
@@ -140,6 +142,33 @@ protected void markAndThrow(Throwable cause, boolean mutateRepaired)
             throw new RuntimeException(e);
     }
 
+    public void verify()

Review Comment:
   Yeah, runs fine for me against J11 w/ `ant testsome -Dtest.name=org.apache.cassandra.tools.nodetool.VerifyTest`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183053472


##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubIterator.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class ScrubIterator extends PartitionIndex.IndexPosIterator implements ScrubPartitionIterator
+{
+    ByteBuffer key;
+    long dataPosition;
+    final FileHandle rowIndexFile;
+
+    ScrubIterator(PartitionIndex partitionIndex, FileHandle rowIndexFile) throws IOException
+    {
+        super(partitionIndex);
+        this.rowIndexFile = rowIndexFile.sharedCopy();
+        advance();
+    }
+
+    @Override
+    public void close()
+    {
+        super.close();
+        rowIndexFile.close();
+    }
+
+    @Override
+    public ByteBuffer key()
+    {
+        return key;
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return dataPosition;
+    }
+
+    @Override
+    public void advance() throws IOException
+    {
+        long pos = nextIndexPos();
+        if (pos != PartitionIndex.NOT_FOUND)
+        {
+            if (pos >= 0) // row index position
+            {
+                try (FileDataInput in = rowIndexFile.createReader(pos))
+                {
+                    key = ByteBufferUtil.readWithShortLength(in);
+                    dataPosition = TrieIndexEntry.deserialize(in, in.getFilePointer()).position;
+                }
+            }
+            else
+            {
+                key = null;
+                dataPosition = ~pos;
+            }
+        }
+        else
+        {
+            key = null;
+            dataPosition = -1;
+        }
+    }
+
+    @Override
+    public boolean isExhausted()
+    {
+        return dataPosition == -1;

Review Comment:
   nit: Obligatory suggestion to use named constant for `-1` :p



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182093686


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            checkArgument(!online || builder.getSerializationHeader() != null);
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings({ "resource", "RedundantSuppression" })
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)

Review Comment:
   nit: Is there actually any way `filter` can be `null` here after it's returned from `buildBloomFilter()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1182095340


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableScanner;
+import org.apache.cassandra.io.util.FileUtils;
+
+public class BtiTableScanner extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>
+{
+    // Full scan of the sstables
+    public static BtiTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static BtiTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        super(sstable, columns, dataRange, rangeIterator, listener);
+    }
+
+    protected void doClose() throws IOException
+    {
+        FileUtils.close(dfile, iterator);
+    }
+
+    protected BtiScanningIterator doCreateIterator()
+    {
+        return new BtiScanningIterator();
+    }
+
+    protected class BtiScanningIterator extends SSTableScanner<BtiTableReader, TrieIndexEntry, BtiTableScanner.BtiScanningIterator>.BaseKeyScanningIterator implements Closeable
+    {
+        private PartitionIterator iterator;
+
+        protected boolean prepareToIterateRow() throws IOException

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1184159452


##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```
   also on 41, 48, and 62



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183602502


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();

Review Comment:
   Added



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,428 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+/**
+ * Writes SSTables in BTI format (see {@link BtiFormat}), which can be read by {@link BtiTableReader}.
+ */
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexBlockCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+    @Override
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    @Override
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page.
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @Override
+    @SuppressWarnings({"resource", "RedundantSuppression"})
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    @Override
+    protected TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy(() -> FBUtilities.immutableListWithFilteredNulls(iwriter, dataWriter));
+    }
+
+    private class TransactionalProxy extends SortedTableWriter<BtiFormatPartitionWriter>.TransactionalProxy
+    {
+        public TransactionalProxy(Supplier<ImmutableList<Transactional>> transactionals)
+        {
+            super(transactionals);
+        }
+
+        @Override
+        protected Throwable doPostCleanup(Throwable accumulate)
+        {
+            accumulate = Throwables.close(accumulate, partitionWriter);
+            accumulate = super.doPostCleanup(accumulate);
+            return accumulate;
+        }
+    }
+
+    /**
+     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
+     */
+    static class IndexWriter extends SortedTableWriter.AbstractIndexWriter
+    {
+        final SequentialWriter rowIndexWriter;
+        private final FileHandle.Builder rowIndexFHBuilder;
+        private final SequentialWriter partitionIndexWriter;
+        private final FileHandle.Builder partitionIndexFHBuilder;
+        private final PartitionIndexBuilder partitionIndex;
+        boolean partitionIndexCompleted = false;
+        private DataPosition riMark;
+        private DataPosition piMark;
+
+        IndexWriter(Builder b)
+        {
+            super(b);
+            rowIndexWriter = new SequentialWriter(descriptor.fileFor(Components.ROW_INDEX), b.getIOOptions().writerOptions);
+            rowIndexFHBuilder = IndexComponent.fileBuilder(Components.ROW_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndexWriter = new SequentialWriter(descriptor.fileFor(Components.PARTITION_INDEX), b.getIOOptions().writerOptions);
+            partitionIndexFHBuilder = IndexComponent.fileBuilder(Components.PARTITION_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndex = new PartitionIndexBuilder(partitionIndexWriter, partitionIndexFHBuilder);
+            // register listeners to be alerted when the data files are flushed
+            partitionIndexWriter.setPostFlushListener(() -> partitionIndex.markPartitionIndexSynced(partitionIndexWriter.getLastFlushOffset()));
+            rowIndexWriter.setPostFlushListener(() -> partitionIndex.markRowIndexSynced(rowIndexWriter.getLastFlushOffset()));
+            @SuppressWarnings({"resource", "RedundantSuppression"})
+            SequentialWriter dataWriter = b.getDataWriter();
+            dataWriter.setPostFlushListener(() -> partitionIndex.markDataSynced(dataWriter.getLastFlushOffset()));
+        }
+
+        public long append(DecoratedKey key, AbstractRowIndexEntry indexEntry) throws IOException
+        {
+            bf.add(key);
+            long position;
+            if (indexEntry.isIndexed())
+            {
+                long indexStart = rowIndexWriter.position();
+                try
+                {
+                    ByteBufferUtil.writeWithShortLength(key.getKey(), rowIndexWriter);
+                    ((TrieIndexEntry) indexEntry).serialize(rowIndexWriter, rowIndexWriter.position());
+                }
+                catch (IOException e)
+                {
+                    throw new FSWriteError(e, rowIndexWriter.getFile());
+                }
+
+                if (logger.isTraceEnabled())
+                    logger.trace("wrote index entry: {} at {}", indexEntry, indexStart);
+                position = indexStart;
+            }
+            else
+            {
+                // Write data position directly in trie.
+                position = ~indexEntry.position;
+            }
+            partitionIndex.addEntry(key, position);
+            return position;
+        }
+
+        public boolean buildPartial(long dataPosition, Consumer<PartitionIndex> callWhenReady)

Review Comment:
   This is okay, we can safely ignore not scheduling another early-open session, but it's still a good idea to report that we haven't.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        @Override
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        @Override
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        @Override
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;

Review Comment:
   Added



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();
+            return partitionIterator;
+        }
+        catch (IOException | RuntimeException ex)
+        {
+            if (partitionIterator != null)
+            {
+                partitionIterator.close();
+            }
+            else
+            {
+                Throwables.closeNonNullAndAddSuppressed(ex, rowIndexFileCopy, dataFileCopy, partitionIndexCopy);
+            }
+            throw ex;
+        }
+    }
+
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile) throws IOException
+    {
+        return create(partitionIndex, partitioner, rowIndexFile, dataFile, partitionIndex.firstKey(), -1, partitionIndex.lastKey(), 0);
+    }
+
+    static PartitionIterator empty(PartitionIndex partitionIndex)
+    {
+        return new PartitionIterator(partitionIndex.sharedCopy());
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                              PartitionPosition left, PartitionPosition right, int exclusiveRight)
+    {
+        super(partitionIndex, left, right);
+        this.partitionIndex = partitionIndex;
+        this.partitioner = partitioner;
+        this.limit = right;
+        this.exclusiveLimit = exclusiveRight;
+        this.rowIndexFile = rowIndexFile;
+        this.dataFile = dataFile;
+    }
+
+    private PartitionIterator(PartitionIndex partitionIndex)
+    {
+        super(partitionIndex, partitionIndex.firstKey(), partitionIndex.firstKey());
+        this.partitionIndex = partitionIndex;
+        this.partitioner = null;
+        this.limit = partitionIndex.firstKey();
+        this.exclusiveLimit = -1;
+        this.rowIndexFile = null;
+        this.dataFile = null;
+
+        this.currentEntry = null;
+        this.currentKey = null;
+        this.nextEntry = null;
+        this.nextKey = null;
+    }
+
+    @Override
+    public void close()
+    {
+        Throwable accum = null;
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(partitionIndex, dataFile, rowIndexFile));
+        accum = Throwables.close(accum, immutableListWithFilteredNulls(dataInput, indexInput));
+        accum = Throwables.perform(accum, super::close);
+        Throwables.maybeFail(accum);
+    }
+
+    public DecoratedKey decoratedKey()
+    {
+        return currentKey;
+    }
+
+    public ByteBuffer key()
+    {
+        return currentKey.getKey();
+    }
+
+    @Override
+    public long dataPosition()
+    {
+        return currentEntry != null ? currentEntry.position : -1;
+    }
+
+    @Override
+    public long keyPositionForSecondaryIndex()
+    {
+        return dataPosition();
+    }
+
+    public TrieIndexEntry entry()
+    {
+        return currentEntry;
+    }
+
+    @Override
+    public boolean advance() throws IOException
+    {
+        currentKey = nextKey;
+        currentEntry = nextEntry;
+        if (currentKey != null)
+        {
+            readNext();
+            // if nextKey is null, then currentKey is the last key to be published, therefore check against any limit
+            // and suppress the partition if it is beyond the limit
+            if (nextKey == null && limit != null && currentKey.compareTo(limit) > exclusiveLimit)
+            {   // exclude last partition outside range
+                currentKey = null;

Review Comment:
   Added



##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+@NotThreadSafe
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        final long node;
+        final int limit;
+        final IterationPosition prev;
+        int childIndex;
+
+        public IterationPosition(long node, int childIndex, int limit, IterationPosition prev)
+        {
+            super();
+            this.node = node;
+            this.childIndex = childIndex;
+            this.limit = limit;
+            this.prev = prev;
+        }
+    }
+
+    protected ReverseValueIterator(Rebufferer source, long root)

Review Comment:
   It's unused and doesn't appear correct. Removed.



##########
src/java/org/apache/cassandra/io/sstable/format/SortedTableVerifier.java:
##########
@@ -140,6 +142,33 @@ protected void markAndThrow(Throwable cause, boolean mutateRepaired)
             throw new RuntimeException(e);
     }
 
+    public void verify()

Review Comment:
   Runs with J8 under IDEA and with `ant test`. I don't want to switch my IDEA target as it always breaks something in my config.



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ *   - calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   - delaying writing any part of a completed node until its branch size is above the page size
+ *   - laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   - adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ *
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ *
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * where lines denote page boundaries.
+ *
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+                child.filePos = write(child);
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+                || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    protected String dumpNode(Node<VALUE> node, long nodePosition)
+    {
+        StringBuilder res = new StringBuilder(String.format("At %,d(%x) type %s child count %s nodeSize %,d branchSize %,d %s%s%n",
+                                                            nodePosition, nodePosition,
+                                                            TrieNode.typeFor(node, nodePosition), node.childCount(), node.nodeSize, node.branchSize,
+                                                            node.hasOutOfPageChildren ? "C" : "",
+                                                            node.hasOutOfPageInBranch ? "B" : ""));
+        for (Node<VALUE> child : node.children)
+            res.append(String.format("Child %2x at %,d(%x) type %s child count %s size %s nodeSize %,d branchSize %,d %s%s%n",
+                                     child.transition & 0xFF,
+                                     child.filePos,
+                                     child.filePos,
+                                     child.children != null ? TrieNode.typeFor(child, child.filePos) : "n/a",
+                                     child.children != null ? child.childCount() : "n/a",
+                                     child.children != null ? serializer.sizeofNode(child, child.filePos) : "n/a",
+                                     child.nodeSize,
+                                     child.branchSize,
+                                     child.hasOutOfPageChildren ? "C" : "",
+                                     child.hasOutOfPageInBranch ? "B" : ""));
+
+        return res.toString();
+    }
+
+    @Override
+    public PartialTail makePartialRoot() throws IOException
+    {
+        // The expectation is that the partial tail will be in memory, so we don't bother with page-fitting.
+        // We could also send some completed children to disk, but that could make suboptimal layout choices, so we'd
+        // rather not. Just write anything not written yet to a buffer, from bottom to top, and we're done.
+        try (DataOutputBuffer buf = new DataOutputBuffer())
+        {
+            PTail tail = new PTail();
+            // Readers ask rebufferers for page-aligned positions, so make sure tail starts at one.
+            // "Padding" of the cutoff point may leave some unaddressable space in the constructed file view.
+            // Nothing will point to it, though, so that's fine.
+            tail.cutoff = dest.paddedPosition();
+            tail.count = count;
+            tail.root = writePartial(stack.getFirst(), buf, tail.cutoff);
+            tail.tail = buf.asNewBuffer();
+            return tail;
+        }
+    }
+
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                child.filePos = writePartial(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    static class Node<Value> extends IncrementalTrieWriterBase.BaseNode<Value, Node<Value>>
+    {
+        /**
+         * Currently calculated size of the branch below this node, not including the node itself.
+         * If hasOutOfPageInBranch is true, this may be underestimated as the size
+         * depends on the position the branch is written.
+         */
+        int branchSize = -1;
+        /**
+         * Currently calculated node size. If hasOutOfPageChildren is true, this may be underestimated as the size
+         * depends on the position the node is written.
+         */
+        int nodeSize = -1;
+
+        /**
+         * Whether there is an out-of-page, already written node in the branches below the immediate children of the
+         * node.
+         */
+        boolean hasOutOfPageInBranch = false;
+        /**
+         * Whether a child of the node is out of page, already written.
+         * Forced to true before being set to make sure maxPositionDelta performs its evaluation on non-completed
+         * nodes for makePartialRoot.
+         */
+        boolean hasOutOfPageChildren = true;
+
+        Node(int transition)
+        {
+            super(transition);
+        }
+
+        @Override
+        Node<Value> newNode(byte transition)
+        {
+            return new Node<>(transition & 0xFF);
+        }
+
+        public long serializedPositionDelta(int i, long nodePosition)
+        {
+            assert (children.get(i).filePos != -1);
+            return children.get(i).filePos - nodePosition;
+        }
+
+        /**
+         * The max delta is the delta with either:
+         * - the position where the first child not-yet-placed child will be laid out.
+         * - the position of the furthest child that is already placed.
+         *

Review Comment:
   Done (with more reformatting)



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ *   - calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   - delaying writing any part of a completed node until its branch size is above the page size
+ *   - laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   - adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ *
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ *
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * where lines denote page boundaries.
+ *
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;

Review Comment:
   This is not reachable. Children are written when their parent is being completed, thus no child can be written at this point (unless there's sharing of branches, which we don't have at this time).
   
   Finding this inspired me to try rewriting `layoutChildren` to avoid the extra `TreeSet` (which creates a bit of garbage we could do without). The result is [here](https://github.com/blambov/cassandra/commit/9b02b25aba9d6ef16a42e339f4de78c48ed2f98b); I am not including it in this PR as it needs some testing to prove it adds value.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        @Override
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        @Override
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        @Override
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+
+            ClusteringBound<?> start = slice.start();
+            long currentPosition = file.getFilePointer();
+            foundLessThan = false;
+            // This is a copy of handlePreSliceData which also checks currentPosition < stopPosition.
+            // Not extracted to method as we need both marker and currentPosition.
+            if (filterStart)
+            {
+                while (currentPosition < stopPosition && deserializer.hasNext() && deserializer.compareNextTo(start) <= 0)
+                {
+                    if (deserializer.nextIsRow())
+                        deserializer.skipNext();
+                    else
+                        updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                    currentPosition = file.getFilePointer();
+                    foundLessThan = true;
+                }
+            }
+
+            // We've reached the beginning of our queried slice. If we have an open marker
+            // we should return that at the end of the slice to close the deletion.
+            if (openMarker != null)
+                blockOpenMarker = new RangeTombstoneBoundMarker(start, openMarker);
+
+
+            // Now deserialize everything until we reach our requested end (if we have one)
+            // See SSTableIterator.ForwardRead.computeNext() for why this is a strict inequality below: this is the same
+            // reasoning here.
+            while (currentPosition < stopPosition && deserializer.hasNext()
+                   && (!filterEnd || deserializer.compareNextTo(slice.end()) < 0))
+            {
+                rowOffsets.push(currentPosition);
+                if (deserializer.nextIsRow())
+                    deserializer.skipNext();
+                else
+                    updateOpenMarker((RangeTombstoneMarker) deserializer.readNext());
+
+                currentPosition = file.getFilePointer();
+            }
+
+            // If we have an open marker, we should output that first, unless end is not being filtered
+            // (i.e. it's either top (where a marker can't be open) or we placed that marker during previous block).
+            if (openMarker != null && filterEnd)
+            {
+                // If we have no end and still an openMarker, this means we're indexed and the marker is closed in a following block.
+                blockCloseMarker = new RangeTombstoneBoundMarker(slice.end(), openMarker);
+                openMarker = null;
+            }
+        }
+    }
+
+    private class ReverseIndexedReader extends ReverseReader
+    {
+        private RowIndexReverseIterator indexReader;
+        private final TrieIndexEntry indexEntry;
+        private final long basePosition;

Review Comment:
   It does not to me -- block positions are calculated as `base + offset`. The offset can (at least in theory) be negative too.



##########
test/unit/org/apache/cassandra/io/tries/TrieBuilderTest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.TailOverridingRebufferer;
+
+import static org.junit.Assert.assertEquals;
+
+public class TrieBuilderTest extends AbstractTrieTestBase
+{
+    @Test
+    public void testPartialBuild_DB1148() throws IOException

Review Comment:
   Renamed



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181732179


##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.
+ * <p>
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public final class TrieIndexEntry extends AbstractRowIndexEntry
+{
+    final long indexTrieRoot;
+    private final int rowIndexBlockCount;
+    private final DeletionTime deletionTime;
+
+    TrieIndexEntry(long dataFilePosition, long indexTrieRoot, int rowIndexBlockCount, DeletionTime deletionTime)
+    {
+        super(dataFilePosition);
+        this.indexTrieRoot = indexTrieRoot;
+        this.rowIndexBlockCount = rowIndexBlockCount;
+        this.deletionTime = deletionTime;
+    }
+
+    public TrieIndexEntry(long position)
+    {
+        super(position);
+        this.indexTrieRoot = -1;
+        this.rowIndexBlockCount = 0;
+        this.deletionTime = null;
+    }
+
+    @Override
+    public int blockCount()
+    {
+        return rowIndexBlockCount;
+    }
+
+    @Override
+    public SSTableFormat<?, ?> getSSTableFormat()
+    {
+        return BtiFormat.instance;
+    }
+
+    @Override
+    public void serializeForCache(DataOutputPlus out)
+    {
+        throw new AssertionError("BTI SSTables should not use key cache");
+    }
+
+    @Override
+    public DeletionTime deletionTime()
+    {
+        return deletionTime;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        throw new AssertionError("BTI SSTables index entries should not be persisted in any in-memory structure");
+    }
+
+    public void serialize(DataOutputPlus indexFile, long basePosition) throws IOException
+    {
+        assert indexTrieRoot != -1 && rowIndexBlockCount > 0 && deletionTime != null;
+        indexFile.writeUnsignedVInt(position);
+        indexFile.writeVInt(indexTrieRoot - basePosition);
+        indexFile.writeUnsignedVInt32(rowIndexBlockCount);
+        DeletionTime.serializer.serialize(deletionTime, indexFile);
+    }
+
+    /**
+     * Create an index entry. The row index trie must already have been written (by RowIndexWriter) to the row index
+     * file and its root position must be specified in trieRoot.
+     */
+    public static TrieIndexEntry create(long dataStartPosition,
+                                        long trieRoot,
+                                        DeletionTime partitionLevelDeletion,
+                                        int rowIndexBlockCount)
+    {
+        return new TrieIndexEntry(dataStartPosition, trieRoot, trieRoot == -1 ? 0 : rowIndexBlockCount, partitionLevelDeletion);
+    }
+
+    public static TrieIndexEntry deserialize(DataInputPlus in, long basePosition) throws IOException
+    {
+        long dataFilePosition = in.readUnsignedVInt();
+        long indexTrieRoot = in.readVInt() + basePosition;
+        int rowIndexCount = in.readUnsignedVInt32();

Review Comment:
   ```suggestion
           int rowIndexBlockCount = in.readUnsignedVInt32();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185315173


##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -163,6 +168,14 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
 
     public static SSTableReader sstable(int generation, int size, boolean keepRef, long firstToken, long lastToken, int level, ColumnFamilyStore cfs, int minLocalDeletionTime, long timestamp)
     {
+        try (DataOutputStreamPlus out = tempFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            out.write(new byte[10]);

Review Comment:
   What does this do?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183084470


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException

Review Comment:
   nit: `@Override`



##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183072939


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();

Review Comment:
   nit: Any test coverage for this path?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185508294


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));

Review Comment:
   ```suggestion
                           assertSame(right, keys.get(idx));
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185506673


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());

Review Comment:
   nit: Clean up commented bits?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1195438942


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {

Review Comment:
   Added



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1196331499


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }

Review Comment:
   The intervals are already adjusted by `SSTableScanner`, which is the only caller.
   
   Removed the adjustments, inlined the method and added to JavaDoc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183000132


##########
src/java/org/apache/cassandra/io/sstable/format/bti/ScrubPartitionIterator.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/**
+ * Iterator over the partitions of an sstable used for scrubbing.
+ * <p>
+ * The difference between this and {@Link PartitionIterator} is that this only uses information present in the index file

Review Comment:
   ```suggestion
    * The difference between this and {@link PartitionIterator} is that this only uses information present in the index file
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179666857


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)

Review Comment:
   ```suggestion
           private int sizeof(IndexInfo payload)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179712914


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")

Review Comment:
   ```suggestion
       @SuppressWarnings({ "resource", "RedundantSuppression" })
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov closed pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov closed pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format
URL: https://github.com/apache/cassandra/pull/2267


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178646875


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.
+            return -1;
+        }
+    }
+
+    protected void addIndexBlock() throws IOException
+    {
+        IndexInfo cIndexInfo = new IndexInfo(startPosition,
+                                             startOpenMarker);
+        rowTrie.add(firstClustering, lastClustering, cIndexInfo);
+        firstClustering = null;
+        ++rowIndexCount;
+    }
+
+    public int getRowIndexCount()

Review Comment:
   nit: If `rowIndexCount` becomes, `blockCount`, rename...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178645294


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;

Review Comment:
   nit: I'd consider naming this `blockSize`, as that's literally what it is?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178646875


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version)
+    {
+        this(header, comparator, dataWriter, rowIndexWriter, version, DatabaseDescriptor.getColumnIndexSize());
+    }
+
+
+    BtiFormatPartitionWriter(SerializationHeader header,
+                             ClusteringComparator comparator,
+                             SequentialWriter dataWriter,
+                             SequentialWriter rowIndexWriter,
+                             Version version,
+                             int indexSize)
+    {
+        super(header, dataWriter, version);
+        this.indexSize = indexSize;
+        this.rowTrie = new RowIndexWriter(comparator, rowIndexWriter);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rowTrie.reset();
+        rowIndexCount = 0;
+    }
+
+    @Override
+    public void addUnfiltered(Unfiltered unfiltered) throws IOException
+    {
+        super.addUnfiltered(unfiltered);
+
+        // if we hit the column index size that we have to index after, go ahead and index it.
+        if (currentPosition() - startPosition >= indexSize)
+            addIndexBlock();
+    }
+
+    @Override
+    public void close()
+    {
+        rowTrie.close();
+    }
+
+    public long finish() throws IOException
+    {
+        long endPosition = super.finish();
+
+        // the last row may have fallen on an index boundary already.  if not, index it explicitly.
+        if (rowIndexCount > 0 && firstClustering != null)
+            addIndexBlock();
+
+        if (rowIndexCount > 1)
+        {
+            return rowTrie.complete(endPosition);
+        }
+        else
+        {
+            // Otherwise we don't complete the trie. Even if we did write something (which shouldn't be the case as the
+            // first entry has an empty key and root isn't filled), that's not a problem.
+            return -1;
+        }
+    }
+
+    protected void addIndexBlock() throws IOException
+    {
+        IndexInfo cIndexInfo = new IndexInfo(startPosition,
+                                             startOpenMarker);
+        rowTrie.add(firstClustering, lastClustering, cIndexInfo);
+        firstClustering = null;
+        ++rowIndexCount;
+    }
+
+    public int getRowIndexCount()

Review Comment:
   nit: If `rowIndexCount` becomes, `blockCount`, rename...
   
   Then there are the callers of this method, which pass the result to things named `rowIndexCount`. Again, this feels wrongly named. There is only one row index, and it has "entries" and "blocks"...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180753851


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ * <ul>
+ *     <li>data file position if the partition is small enough to not need an index
+ *     <li>row index file position if the partition has a row index
+ * </ul>plus<ul>
+ *     <li>the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ * </ul>
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ * <p>
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ * <p>
+ * The indexes are created by {@link PartitionIndexBuilder}. To read the index one must obtain a thread-unsafe
+ * {@link Reader} or {@link IndexPosIterator}.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;

Review Comment:
   nit: Brief message hinting at why this might happen?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180829911


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;

Review Comment:
   nit: Do we need this, given we never use it anyway? ...or does it provide some sort of protection against aberrant usage?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162091485


##########
src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java:
##########
@@ -337,6 +337,41 @@ private void seekToChunkStart()
         }
     }
 
+    // Page management using chunk boundaries
+
+    @Override
+    public int maxBytesInPage()
+    {
+        return buffer.capacity();
+    }
+
+    @Override
+    public void padToPageBoundary() throws IOException

Review Comment:
   nit:
   ```suggestion
       public void padToPageBoundary()
   ```
   (I think we removed it in the SAI patch as well.)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1164659034


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();

Review Comment:
   nit: Line 60 to the end of the method is duplicated in `BigTableVerifier#verify()` FWIW



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167207958


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *

Review Comment:
   ```suggestion
            * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167286657


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException

Review Comment:
   nit: Unless we specifically want to document why we'd throw IOE, I guess we can remove this, as it's already in the method signature.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167168575


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReverseIterator.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.PrintStream;
+
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.ReverseValueIterator;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reverse iterator over the row index. Needed to get previous index blocks for reverse iteration.
+ */
+class RowIndexReverseIterator extends ReverseValueIterator<RowIndexReverseIterator>
+{
+    private long currentNode = -1;
+
+    public RowIndexReverseIterator(FileHandle file, long root, ByteComparable start, ByteComparable end)
+    {
+        super(file.instantiateRebufferer(null), root, start, end, true);
+    }
+
+    public RowIndexReverseIterator(FileHandle file, TrieIndexEntry entry, ByteComparable end)
+    {
+        this(file, entry.indexTrieRoot, ByteComparable.EMPTY, end);
+    }
+
+    /**
+     * This method must be async-read-safe.
+     */
+    public IndexInfo nextIndexInfo()
+    {
+        if (currentNode == -1)
+        {
+            currentNode = nextPayloadedNode();
+            if (currentNode == -1)
+                return null;
+        }
+
+        go(currentNode);
+        IndexInfo info = RowIndexReader.readPayload(buf, payloadPosition(), payloadFlags());
+
+        currentNode = -1;
+        return info;
+    }
+
+    public void dumpTrie(PrintStream out)

Review Comment:
   nit: unused (although it looks like it's mostly here for testing/diagnostics, so not a big deal)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167193866


##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;

Review Comment:
   nit: `limit` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167185112


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;

Review Comment:
   nit: the block from line 131-151 looks like it's more or less duplicated in `IncrementalTrieWriterPageAware#writePartial()` and `WritePartialRecursion#complete()`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167152309


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);
+
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);

Review Comment:
   Not directly related to this patch, but if you go all the way up the hierarchy, `SSTable#unbuildTo()` doesn't use its `boolean sharedCopy` argument :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171635094


##########
src/java/org/apache/cassandra/io/tries/TrieNode.java:
##########
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SizedInts;
+
+/**
+ * Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as
+ * they are on disk without any serialization, and to enable the creation of such files.
+ * <p>
+ * The serialization methods take as argument a generic {@code SerializationNode} and provide a method {@code typeFor}
+ * for choosing a suitable type to represent it, which can then be used to calculate size and write the node.
+ * <p>
+ * To read a file containing trie nodes, one would use {@code at} to identify the node type and then the various
+ * read methods to retrieve the data. They all take a buffer (usually memory-mapped) containing the data, and a position
+ * in it that identifies the node.
+ * <p>
+ * These node types do not specify any treatment of payloads. They are only concerned with providing 4 bits of
+ * space for {@code payloadFlags}, and a way of calculating the position after the node. Users of this class by convention
+ * use non-zero payloadFlags to indicate a payload exists, write it (possibly in flag-dependent format) at serialization
+ * time after the node itself is written, and read it using the {@code payloadPosition} value.
+ * <p>
+ * To improve efficiency, multiple node types depending on the number of transitions are provided:
+ * -- payload only, which has no outgoing transitions
+ * -- single outgoing transition
+ * -- sparse, which provides a list of transition bytes with corresponding targets
+ * -- dense, where the transitions span a range of values and having the list (and the search in it) can be avoided
+ * <p>
+ * For each of the transition-carrying types we also have "in-page" versions where transition targets are the 4, 8 or 12
+ * lowest bits of the position within the same page. To save one further byte, the single in-page versions using 4 or 12
+ * bits cannot carry a payload.
+ * <p>
+ * This class is effectively an enumeration; abstract class permits instances to extend each other and reuse code.
+ */
+public abstract class TrieNode
+{
+    // Consumption (read) methods
+
+    /**
+     * Returns the type of node stored at this position. It can then be used to call the methods below.
+     */
+    public static TrieNode at(ByteBuffer src, int position)
+    {
+        return values[(src.get(position) >> 4) & 0xF];
+    }
+
+    /**
+     * Returns the 4 payload flag bits. Node types that cannot carry a payload return 0.
+     */
+    public int payloadFlags(ByteBuffer src, int position)
+    {
+        return src.get(position) & 0x0F;
+    }
+
+    /**
+     * Return the position just after the node, where the payload is usually stored.
+     */
+    abstract public int payloadPosition(ByteBuffer src, int position);
+
+    /**
+     * Returns search index for the given byte in the node. If exact match is present, this is >= 0, otherwise as in
+     * binary search.
+     */
+    abstract public int search(ByteBuffer src, int position, int transitionByte);       // returns as binarySearch
+
+    /**
+     * Returns the upper childIndex limit. Calling transition with values 0...transitionRange - 1 is valid.
+     */
+    abstract public int transitionRange(ByteBuffer src, int position);
+
+    /**
+     * Returns the byte value for this child index, or Integer.MAX_VALUE if there are no transitions with this index or
+     * higher to permit listing the children without needing to call transitionRange.
+     *
+     * @param childIndex must be >= 0, though it is allowed to pass a value greater than {@code transitionRange - 1}
+     */
+    abstract public int transitionByte(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns the delta between the position of this node and the position of the target of the specified transition.
+     * This is always a negative number. Dense nodes use 0 to specify "no transition".
+     *
+     * @param childIndex must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                   and behaviour of this method is undefined for values outside of that range
+     */
+    abstract long transitionDelta(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns position of node to transition to for the given search index. Argument must be positive. May return -1
+     * if a transition with that index does not exist (DENSE nodes).
+     * Position is the offset of the node within the ByteBuffer. positionLong is its global placement, which is the
+     * base for any offset calculations.
+     *
+     * @param positionLong although it seems to be obvious, this argument must be "real", that is, each child must have
+     *                     the calculated absolute position >= 0, otherwise the behaviour of this method is undefined
+     * @param childIndex   must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                     and behaviour of this method is undefined for values outside of that range
+     */
+    public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+    {
+        // note: this is not valid for dense nodes
+        return positionLong + transitionDelta(src, position, childIndex);
+    }
+
+    /**
+     * Returns the highest transition for this node, or -1 if none exist (PAYLOAD_ONLY nodes).
+     */
+    public long lastTransition(ByteBuffer src, int position, long positionLong)
+    {
+        return transition(src, position, positionLong, transitionRange(src, position) - 1);
+    }
+
+    /**
+     * Returns a transition that is higher than the index returned by {@code search}. This may not exist (if the
+     * argument was higher than the last transition byte), in which case this returns the given {@code defaultValue}.
+     */
+    abstract public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    /**
+     * Returns a transition that is lower than the index returned by {@code search}. Returns {@code defaultValue} for
+     * {@code searchIndex} equals 0 or -1 as lesser transition for those indexes does not exist.
+     */
+    abstract public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    // Construction (serialization) methods
+
+    /**
+     * Returns a node type that is suitable to store the node.
+     */
+    public static TrieNode typeFor(SerializationNode<?> node, long nodePosition)
+    {
+        int c = node.childCount();
+        if (c == 0)
+            return PAYLOAD_ONLY;
+
+        int bitsPerPointerIndex = 0;
+        long delta = node.maxPositionDelta(nodePosition);
+        assert delta < 0;
+        while (!singles[bitsPerPointerIndex].fits(-delta))
+            ++bitsPerPointerIndex;
+
+        if (c == 1)
+        {
+            if (node.payload() != null && singles[bitsPerPointerIndex].bytesPerPointer == FRACTIONAL_BYTES)
+                ++bitsPerPointerIndex; // next index will permit payload
+
+            return singles[bitsPerPointerIndex];
+        }
+
+        TrieNode sparse = sparses[bitsPerPointerIndex];
+        TrieNode dense = denses[bitsPerPointerIndex];
+        return (sparse.sizeofNode(node) < dense.sizeofNode(node)) ? sparse : dense;
+    }
+
+    /**
+     * Returns the size needed to serialize this node.
+     */
+    abstract public int sizeofNode(SerializationNode<?> node);
+
+    /**
+     * Serializes the node. All transition target positions must already have been defined. {@code payloadBits} must
+     * be four bits.
+     */
+    abstract public void serialize(DataOutputPlus out, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException;
+
+    // Implementations
+
+    final int bytesPerPointer;
+    static final int FRACTIONAL_BYTES = 0;
+
+    TrieNode(int ordinal, int bytesPerPointer)
+    {
+        this.ordinal = ordinal;
+        this.bytesPerPointer = bytesPerPointer;
+    }
+
+    final int ordinal;
+
+    static final TrieNode PAYLOAD_ONLY = new PayloadOnly();
+
+    static private class PayloadOnly extends TrieNode
+    {
+        // byte flags
+        // var payload
+        PayloadOnly()
+        {
+            super(0, FRACTIONAL_BYTES);
+        }
+
+        @Override
+        public int payloadPosition(ByteBuffer src, int position)
+        {
+            return position + 1;
+        }
+
+        @Override
+        public int search(ByteBuffer src, int position, int transitionByte)
+        {
+            return -1;
+        }
+
+        @Override
+        public long transitionDelta(ByteBuffer src, int position, int childIndex)
+        {
+            return 0;
+        }
+
+        @Override
+        public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+        {
+            return -1;
+        }
+
+        @Override
+        public long lastTransition(ByteBuffer src, int position, long positionLong)
+        {
+            return -1;
+        }
+
+        @Override
+        public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public int transitionByte(ByteBuffer src, int position, int childIndex)
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public int transitionRange(ByteBuffer src, int position)
+        {
+            return 0;
+        }
+
+        public int sizeofNode(SerializationNode<?> node)
+        {
+            return 1;
+        }
+
+        @Override
+        public void serialize(DataOutputPlus dest, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException
+        {
+            dest.writeByte((ordinal << 4) + (payloadBits & 0x0F));
+        }
+    }
+
+    static final TrieNode SINGLE_8 = new Single(2, 1);
+    static final TrieNode SINGLE_16 = new Single(4, 2);

Review Comment:
   hmmm, IDEA's static analysis tooling still complains about it...perhaps just a bug there



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171601281


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public class BtiTableScanner implements ISSTableScanner
+{
+    private final AtomicBoolean isClosed = new AtomicBoolean(false);
+    protected final RandomAccessReader dfile;
+    public final BtiTableReader sstable;
+
+    private final Iterator<AbstractBounds<PartitionPosition>> rangeIterator;
+
+    private final ColumnFilter columns;
+    private final DataRange dataRange;
+    private final SSTableReadsListener listener;
+    private long startScan = -1;
+    private long bytesScanned = 0;
+
+    protected CloseableIterator<UnfilteredRowIterator> iterator;
+
+    // Full scan of the sstables
+    public static ISSTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        assert sstable != null;
+
+        this.dfile = sstable.openDataReader();
+        this.sstable = sstable;
+        this.columns = columns;
+        this.dataRange = dataRange;
+        this.rangeIterator = rangeIterator;
+        this.listener = listener;
+    }
+
+    public static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(tokenRanges.size());
+        for (Range<Token> range : Range.normalize(tokenRanges))
+            addRange(sstable, Range.makeRowRange(range), boundsList);
+        return boundsList;
+    }
+
+    static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, DataRange dataRange)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(2);
+        addRange(sstable, dataRange.keyRange(), boundsList);
+        return boundsList;
+    }
+
+    static AbstractBounds<PartitionPosition> fullRange(SSTableReader sstable)
+    {
+        return new Bounds<>(sstable.first, sstable.last);
+    }
+
+    private static void addRange(SSTableReader sstable, AbstractBounds<PartitionPosition> requested, List<AbstractBounds<PartitionPosition>> boundsList)
+    {
+        if (requested instanceof Range && ((Range<?>) requested).isWrapAround())
+        {
+            if (requested.right.compareTo(sstable.first) >= 0)
+            {
+                // since we wrap, we must contain the whole sstable prior to stopKey()
+                Boundary<PartitionPosition> left = new Boundary<>(sstable.first, true);
+                Boundary<PartitionPosition> right;
+                right = requested.rightBoundary();
+                right = minRight(right, sstable.last, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+            if (requested.left.compareTo(sstable.last) <= 0)
+            {
+                // since we wrap, we must contain the whole sstable after dataRange.startKey()
+                Boundary<PartitionPosition> right = new Boundary<>(sstable.last, true);
+                Boundary<PartitionPosition> left;
+                left = requested.leftBoundary();
+                left = maxLeft(left, sstable.first, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+        }
+        else
+        {
+            assert !AbstractBounds.strictlyWrapsAround(requested.left, requested.right);
+            Boundary<PartitionPosition> left, right;
+            left = requested.leftBoundary();
+            right = requested.rightBoundary();
+            left = maxLeft(left, sstable.first, true);
+            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
+            right = requested.right.isMinimum() ? new Boundary<>(sstable.last, true)
+                                                : minRight(right, sstable.last, true);
+            if (!isEmpty(left, right))
+                boundsList.add(AbstractBounds.bounds(left, right));
+        }
+    }
+
+    public void close()
+    {
+        try
+        {
+            if (isClosed.compareAndSet(false, true))
+            {
+                FileUtils.close(dfile);
+                if (iterator != null)
+                    iterator.close();
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    public long getBytesScanned()
+    {
+        return bytesScanned;
+    }
+
+    @Override
+    public long getLengthInBytes()
+    {
+        return sstable.uncompressedLength();
+    }
+
+
+    public long getCompressedLengthInBytes()
+    {
+        return sstable.onDiskLength();
+    }
+
+    @Override
+    public long getCurrentPosition()
+    {
+        return dfile.getFilePointer();
+    }
+
+    @Override
+    public Set<SSTableReader> getBackingSSTables()
+    {
+        return ImmutableSet.of(sstable);
+    }
+
+    public int level()
+    {
+        return sstable.getSSTableLevel();
+    }
+
+    public TableMetadata metadata()
+    {
+        return sstable.metadata();
+    }
+
+    public boolean hasNext()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.hasNext();
+    }
+
+    public UnfilteredRowIterator next()
+    {
+        if (iterator == null)
+            iterator = createIterator();
+        return iterator.next();
+    }
+
+    public void remove()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    private CloseableIterator<UnfilteredRowIterator> createIterator()
+    {
+        this.listener.onScanningStarted(sstable);
+        return new KeyScanningIterator();
+    }
+
+    protected class KeyScanningIterator extends AbstractIterator<UnfilteredRowIterator> implements CloseableIterator<UnfilteredRowIterator>
+    {
+        private DecoratedKey currentKey;
+        private TrieIndexEntry currentEntry;
+        private PartitionIterator iterator;
+        private LazilyInitializedUnfilteredRowIterator currentRowIterator;

Review Comment:
   This was part of code to guard against use of row iterator after advancing the partition one.
   
   This code has now been brought forward, including the related test, and these two classes refactored to remove the repetition.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1172499987


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException
+     */
+    public static boolean equalsWithShortLength(FileDataInput in, ByteBuffer toMatch) throws IOException

Review Comment:
   The `test-trie` target would be the one exercising it.
   
   Added a short direct test.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1567170863

   [Latest run](https://app.circleci.com/pipelines/github/blambov/cassandra/457/workflows/1cd1cf09-2c88-45a2-9f3c-8eb30ffb7f31) looks pretty good. Committing.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1169972040


##########
src/java/org/apache/cassandra/io/tries/TrieNode.java:
##########
@@ -0,0 +1,993 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.SizedInts;
+
+/**
+ * Trie node types and manipulation mechanisms. The main purpose of this is to allow for handling tries directly as
+ * they are on disk without any serialization, and to enable the creation of such files.
+ * <p>
+ * The serialization methods take as argument a generic {@code SerializationNode} and provide a method {@code typeFor}
+ * for choosing a suitable type to represent it, which can then be used to calculate size and write the node.
+ * <p>
+ * To read a file containing trie nodes, one would use {@code at} to identify the node type and then the various
+ * read methods to retrieve the data. They all take a buffer (usually memory-mapped) containing the data, and a position
+ * in it that identifies the node.
+ * <p>
+ * These node types do not specify any treatment of payloads. They are only concerned with providing 4 bits of
+ * space for {@code payloadFlags}, and a way of calculating the position after the node. Users of this class by convention
+ * use non-zero payloadFlags to indicate a payload exists, write it (possibly in flag-dependent format) at serialization
+ * time after the node itself is written, and read it using the {@code payloadPosition} value.
+ * <p>
+ * To improve efficiency, multiple node types depending on the number of transitions are provided:
+ * -- payload only, which has no outgoing transitions
+ * -- single outgoing transition
+ * -- sparse, which provides a list of transition bytes with corresponding targets
+ * -- dense, where the transitions span a range of values and having the list (and the search in it) can be avoided
+ * <p>
+ * For each of the transition-carrying types we also have "in-page" versions where transition targets are the 4, 8 or 12
+ * lowest bits of the position within the same page. To save one further byte, the single in-page versions using 4 or 12
+ * bits cannot carry a payload.
+ * <p>
+ * This class is effectively an enumeration; abstract class permits instances to extend each other and reuse code.
+ */
+public abstract class TrieNode
+{
+    // Consumption (read) methods
+
+    /**
+     * Returns the type of node stored at this position. It can then be used to call the methods below.
+     */
+    public static TrieNode at(ByteBuffer src, int position)
+    {
+        return values[(src.get(position) >> 4) & 0xF];
+    }
+
+    /**
+     * Returns the 4 payload flag bits. Node types that cannot carry a payload return 0.
+     */
+    public int payloadFlags(ByteBuffer src, int position)
+    {
+        return src.get(position) & 0x0F;
+    }
+
+    /**
+     * Return the position just after the node, where the payload is usually stored.
+     */
+    abstract public int payloadPosition(ByteBuffer src, int position);
+
+    /**
+     * Returns search index for the given byte in the node. If exact match is present, this is >= 0, otherwise as in
+     * binary search.
+     */
+    abstract public int search(ByteBuffer src, int position, int transitionByte);       // returns as binarySearch
+
+    /**
+     * Returns the upper childIndex limit. Calling transition with values 0...transitionRange - 1 is valid.
+     */
+    abstract public int transitionRange(ByteBuffer src, int position);
+
+    /**
+     * Returns the byte value for this child index, or Integer.MAX_VALUE if there are no transitions with this index or
+     * higher to permit listing the children without needing to call transitionRange.
+     *
+     * @param childIndex must be >= 0, though it is allowed to pass a value greater than {@code transitionRange - 1}
+     */
+    abstract public int transitionByte(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns the delta between the position of this node and the position of the target of the specified transition.
+     * This is always a negative number. Dense nodes use 0 to specify "no transition".
+     *
+     * @param childIndex must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                   and behaviour of this method is undefined for values outside of that range
+     */
+    abstract long transitionDelta(ByteBuffer src, int position, int childIndex);
+
+    /**
+     * Returns position of node to transition to for the given search index. Argument must be positive. May return -1
+     * if a transition with that index does not exist (DENSE nodes).
+     * Position is the offset of the node within the ByteBuffer. positionLong is its global placement, which is the
+     * base for any offset calculations.
+     *
+     * @param positionLong although it seems to be obvious, this argument must be "real", that is, each child must have
+     *                     the calculated absolute position >= 0, otherwise the behaviour of this method is undefined
+     * @param childIndex   must be >= 0 and < {@link #transitionRange(ByteBuffer, int)} - note that this is not validated
+     *                     and behaviour of this method is undefined for values outside of that range
+     */
+    public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+    {
+        // note: this is not valid for dense nodes
+        return positionLong + transitionDelta(src, position, childIndex);
+    }
+
+    /**
+     * Returns the highest transition for this node, or -1 if none exist (PAYLOAD_ONLY nodes).
+     */
+    public long lastTransition(ByteBuffer src, int position, long positionLong)
+    {
+        return transition(src, position, positionLong, transitionRange(src, position) - 1);
+    }
+
+    /**
+     * Returns a transition that is higher than the index returned by {@code search}. This may not exist (if the
+     * argument was higher than the last transition byte), in which case this returns the given {@code defaultValue}.
+     */
+    abstract public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    /**
+     * Returns a transition that is lower than the index returned by {@code search}. Returns {@code defaultValue} for
+     * {@code searchIndex} equals 0 or -1 as lesser transition for those indexes does not exist.
+     */
+    abstract public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue);
+
+    // Construction (serialization) methods
+
+    /**
+     * Returns a node type that is suitable to store the node.
+     */
+    public static TrieNode typeFor(SerializationNode<?> node, long nodePosition)
+    {
+        int c = node.childCount();
+        if (c == 0)
+            return PAYLOAD_ONLY;
+
+        int bitsPerPointerIndex = 0;
+        long delta = node.maxPositionDelta(nodePosition);
+        assert delta < 0;
+        while (!singles[bitsPerPointerIndex].fits(-delta))
+            ++bitsPerPointerIndex;
+
+        if (c == 1)
+        {
+            if (node.payload() != null && singles[bitsPerPointerIndex].bytesPerPointer == FRACTIONAL_BYTES)
+                ++bitsPerPointerIndex; // next index will permit payload
+
+            return singles[bitsPerPointerIndex];
+        }
+
+        TrieNode sparse = sparses[bitsPerPointerIndex];
+        TrieNode dense = denses[bitsPerPointerIndex];
+        return (sparse.sizeofNode(node) < dense.sizeofNode(node)) ? sparse : dense;
+    }
+
+    /**
+     * Returns the size needed to serialize this node.
+     */
+    abstract public int sizeofNode(SerializationNode<?> node);
+
+    /**
+     * Serializes the node. All transition target positions must already have been defined. {@code payloadBits} must
+     * be four bits.
+     */
+    abstract public void serialize(DataOutputPlus out, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException;
+
+    // Implementations
+
+    final int bytesPerPointer;
+    static final int FRACTIONAL_BYTES = 0;
+
+    TrieNode(int ordinal, int bytesPerPointer)
+    {
+        this.ordinal = ordinal;
+        this.bytesPerPointer = bytesPerPointer;
+    }
+
+    final int ordinal;
+
+    static final TrieNode PAYLOAD_ONLY = new PayloadOnly();
+
+    static private class PayloadOnly extends TrieNode
+    {
+        // byte flags
+        // var payload
+        PayloadOnly()
+        {
+            super(0, FRACTIONAL_BYTES);
+        }
+
+        @Override
+        public int payloadPosition(ByteBuffer src, int position)
+        {
+            return position + 1;
+        }
+
+        @Override
+        public int search(ByteBuffer src, int position, int transitionByte)
+        {
+            return -1;
+        }
+
+        @Override
+        public long transitionDelta(ByteBuffer src, int position, int childIndex)
+        {
+            return 0;
+        }
+
+        @Override
+        public long transition(ByteBuffer src, int position, long positionLong, int childIndex)
+        {
+            return -1;
+        }
+
+        @Override
+        public long lastTransition(ByteBuffer src, int position, long positionLong)
+        {
+            return -1;
+        }
+
+        @Override
+        public long greaterTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public long lesserTransition(ByteBuffer src, int position, long positionLong, int searchIndex, long defaultValue)
+        {
+            return defaultValue;
+        }
+
+        @Override
+        public int transitionByte(ByteBuffer src, int position, int childIndex)
+        {
+            return Integer.MAX_VALUE;
+        }
+
+        @Override
+        public int transitionRange(ByteBuffer src, int position)
+        {
+            return 0;
+        }
+
+        public int sizeofNode(SerializationNode<?> node)
+        {
+            return 1;
+        }
+
+        @Override
+        public void serialize(DataOutputPlus dest, SerializationNode<?> node, int payloadBits, long nodePosition) throws IOException
+        {
+            dest.writeByte((ordinal << 4) + (payloadBits & 0x0F));
+        }
+    }
+
+    static final TrieNode SINGLE_8 = new Single(2, 1);
+    static final TrieNode SINGLE_16 = new Single(4, 2);

Review Comment:
   As the subclasses are now private (see https://github.com/apache/cassandra/pull/2267/commits/8d5de21b48624301875a31d4e35bbc3bf8c1a68f), there is no possibility for concurrent load of class and subclass and hence no deadlock.
   
   The construction was also changed somewhat to make the ordinal value final.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1171204714


##########
src/java/org/apache/cassandra/utils/ByteBufferUtil.java:
##########
@@ -866,4 +869,27 @@ private static boolean startsWith(ByteBuffer src, ByteBuffer prefix, int offset)
 
         return true;
     }
-}
+
+    /**
+     * Returns true if the buffer at the current position in the input matches given buffer.
+     * If true, the input is positioned at the end of the consumed buffer.
+     * If false, the position of the input is undefined.
+     * <p>
+     * The matched buffer is unchanged
+     *
+     * @throws IOException
+     */
+    public static boolean equalsWithShortLength(FileDataInput in, ByteBuffer toMatch) throws IOException

Review Comment:
   The exception path? Would that add any value?



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);
+
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new UnsupportedOperationException("Unsupported op: " + operator);

Review Comment:
   The cases above cover all possible values of `operator`. Changed to `InvalidArgumentException`.



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterSimple.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builder of on-disk tries. Takes sorted input.
+ * <p>
+ * Incremental building is done by maintaining a stack of nodes in progress which follows the path to reach the last
+ * added entry. When a new entry is needed, comparison with the previous can tell us how much of the parents stack
+ * remains the same. The rest of the stack is complete as no new entry can affect them due to the input sorting.
+ * The completed nodes can be written to disk and discarded, keeping only a pointer to their location in the file
+ * (this pointer will be discarded too when the parent node is completed). This ensures that a very limited amount of
+ * data is kept in memory at all times.
+ * <p>
+ * Note: This class is currently unused and stands only as form of documentation for {@link IncrementalTrieWriterPageAware}.

Review Comment:
   Its place is here; the documentation value is non-existent if it is not presented together with the class that builds upon it.
   
   Changed docs somewhat to reference included documentation.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+  
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size_in_kb`

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/TrieIndexFormatUtil.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.utils.FilterFactory;
+
+import static org.apache.cassandra.service.ActiveRepairService.UNREPAIRED_SSTABLE;
+
+public class TrieIndexFormatUtil

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+  
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size_in_kb`
+`cassandra.yaml` parameter).
+
+Our row index implementation thus creates a map from clustering keys or
+prefixes to the data position at the start of the index block which is
+the earliest that could contain a key equal or greater than the given
+one. Additionally, if there is an active deletion at the beginning of
+the block, the index must specify it so that it can be taken into
+account when merging data from multiple sstables.
+
+Each index block will contain at least one key, but generally it will
+have different first and last keys. We don't store these keys, but
+instead we index the positions between blocks by storing a "separator",
+some key that is greater than the last key of the previous block and
+smaller than or equal to the first key of the next[^4]. Then, when we
+look up a given clustering, we follow its bytes as long as we can in the
+trie and we can be certain that all blocks before the closest
+less-than-or-equal entry in the trie cannot contain any data that is
+greater than or equal to the given key.
+
+[^4]: Another way to interpret this is that we index the start of each
+block only, but for efficiency we don't use the first key of the
+block as its beginning, but instead something closer to the last key
+of the previous block (but still greater than it).
+
+It may happen that the identified block actually doesn't contain any
+matching data (e.g. because the looked-up key ends up between the last
+key in the block and the separator), but this only affects efficiency as
+the iteration mechanism does not expect the data position returned by
+the index to be guaranteed to start with elements that fit the criteria;
+it would only have to walk a whole block forward to find the matching
+key.
+
+It is important to keep the number of these false positives low, and at
+the same time we aim for the smallest possible size of the index for a
+given granularity. The choice of separator affects this balance[^5]; the
+option we use, as a good tradeoff in the vein of the unique prefix
+approach used in the partition index, is to use the shortest prefix of
+the next block's beginning key that separates it from the previous
+block's end key, adjusted so that the last byte of it is 1 greater than
+that end key.
+
+[^5]: For example, the best separator for false positives is the next
+possible byte sequence after the previous block's final key, which
+is obtained by adding a 00 byte to its end. This, however, means all
+the bytes of the byte-ordered representation of this key must be
+present in the index, which inflates the index's size and lookup
+complexity.
+
+For example, if block 2 covers "something" to "somewhere" and block 3
+&mdash; "sorry" to "tease", then the sequence "son" is used as the separator
+between blocks 2 and 3. This leaves things like "sommelier" in the area
+that triggers false positives, but stores and has to walk just three
+bytes to find the starting point for iteration.
+
+### Efficiency
+
+Finding the candidate block in the trie involves walking the byte
+ordered representation of the clustering key in the trie and finding the
+closest less-than-or-equal value. The number of steps is proportional to
+the length of the separators &mdash; the lower their number the shorter that
+sequence is, though we can't expect _O_(log _n_) complexity since there may
+be many items sharing the same long prefixes (e.g. if there are long
+strings in the components of the clustering keys before the last). Even
+so, such repeating prefixes are addressed very well by the page-packing
+and `SINGLE_NOPAYLOAD_4` node type, resulting in very efficient walks.
+
+After this step we also perform a linear walk within the data file to
+find the actual start of the matching data. This is usually costlier and
+may involve object allocation and deserialization.
+
+The tradeoff between the size of the index and the time it takes to find
+the relevant rows is controlled by the index granularity. The lower it
+is, the more efficient lookup (especially exact match lookup) becomes at
+the expense of bigger index size. The 16kb default is chosen pretty
+conservatively[^6]; if users don't mind bigger indices something like 4,
+2 or 1kb granularity should be quite a bit more efficient. It is also
+possible to index every row by choosing a granularity of 0kb; at these
+settings in-cache trie-indexed sstables tend to outperform
+`ConcurrentSkipListMap` memtables for reads.
+
+[^6]: This was chosen with the aim to match the size of the trie index
+compared to the earlier version of the row index at its default
+granularity of 64kb.
+
+### Reverse lookup
+
+To perform a reverse lookup, we can use the same mechanism as above
+(with greater-than-or-equal) to find the initial block for the
+iteration. However, in the forward direction we could simply walk the
+data file to find the next rows, but this isn't possible going
+backwards.
+
+To solve this problem the index helps the iteration machinery by
+providing an iterator of index blocks in reverse order. For each index
+block the iteration walks it forward and creates a stack of all its row
+positions, then starts issuing rows by popping and examining rows from
+that stack. When the stack is exhausted it requests the previous block
+from the index and applies the same procedure there.
+
+# Code structure
+
+The implementation is mostly in two packages, `o.a.c.io.tries` contains
+the generic code to construct and read on-disk tries, and 
+`o.a.c.io.sstable.format.bti`, which implements the specifics of the
+format and the two indexes.
+
+## Building tries
+
+Tries are built from sorted keys using an [`IncrementalTrieWriter`](../../../tries/IncrementalTrieWriter.java). 
+The code contains three implementations with increasing complexity:
+- [`IncrementalTrieWriterSimple`](../../../tries/IncrementalTrieWriterSimple.java)
+  implements simple incremental construction of tries from sorted input,
+- [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalTrieWriterPageAware.java)
+  adds packing of nodes to disk pages,
+- [`IncrementalDeepTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java)
+  adds the ability to transition to on-heap recursion for all stages of the construction
+  process to be able to handle very large keys.
+
+Only the latter is used, but we provide (and test) the other two as a form of
+documentation.
+
+The builders take a `TrieSerializer` as parameter, which determines how the nodes
+are written. The indexes implement this using `TrieNode`, writing any payload they
+need immediately after the node serialization.
+
+## Reading tries
+
+The BTI format tries are used directly in their on-disk format. To achieve this,
+all node types are implemented as static objects in `TrieNode`. Reading nodes in
+a file is encapsulated in [`Walker`](../../../tries/Walker.java), 
+which provides a method to `go` to a specific node and use it, i.e. 
+get any associated data, search in the children list and
+follow transitions to children. It also provides functionality to find the
+mapping for a given key, floors and ceilings as well as some combinations.
+Iterating the payloads between two key bounds is implemented by 
+[`ValueIterator`](../../../tries/ValueIterator.java),
+and [`ReverseValueIterator`](../../../tries/ReverseValueIterator.java).
+
+Special care is given to prefixes to make sure the semantics of searches matches
+what the format needs.
+
+## SSTable format implementation
+
+The two indexes are implemented, respectively, by [`PartitionIndex`](PartitionIndex.java)
+and [`RowIndexReader`](RowIndexReader.java)/[`RowIndexWriter`](RowIndexWriter.java). 
+The format implementation extends the filtered
+base class and follows the structure of the BIG implementation, where
+all references to the primary index are replaced with calls to these two 
+classes.
+
+# Index file format in BTI
+
+## Trie nodes
+Implemented in [`TrieNode.java`](../../../tries/TrieNode.java)
+
+Nodes start with four bits of node type, followed by 4 payload bits
+(_pb_), which are 0 if the node has no associated payload; otherwise the
+node type gives an option to compute the starting position for the
+payload (_ppos_). The layout of the node depends on its type.
+
+`PAYLOAD_ONLY` nodes:
+
+-   4 type bits, 0
+
+-   4 payload bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _node_ + 1

Review Comment:
   Changed to _npos_ (with added definition).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1170988693


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScanner.java:
##########
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterators;
+
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.filter.ClusteringIndexFilter;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.AbstractBounds.Boundary;
+import org.apache.cassandra.dht.Bounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.utils.AbstractIterator;
+import org.apache.cassandra.utils.CloseableIterator;
+
+import static org.apache.cassandra.dht.AbstractBounds.isEmpty;
+import static org.apache.cassandra.dht.AbstractBounds.maxLeft;
+import static org.apache.cassandra.dht.AbstractBounds.minRight;
+
+public class BtiTableScanner implements ISSTableScanner
+{
+    private final AtomicBoolean isClosed = new AtomicBoolean(false);
+    protected final RandomAccessReader dfile;
+    public final BtiTableReader sstable;
+
+    private final Iterator<AbstractBounds<PartitionPosition>> rangeIterator;
+
+    private final ColumnFilter columns;
+    private final DataRange dataRange;
+    private final SSTableReadsListener listener;
+    private long startScan = -1;
+    private long bytesScanned = 0;
+
+    protected CloseableIterator<UnfilteredRowIterator> iterator;
+
+    // Full scan of the sstables
+    public static ISSTableScanner getScanner(BtiTableReader sstable)
+    {
+        return getScanner(sstable, Iterators.singletonIterator(fullRange(sstable)));
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable,
+                                             ColumnFilter columns,
+                                             DataRange dataRange,
+                                             SSTableReadsListener listener)
+    {
+        return new BtiTableScanner(sstable, columns, dataRange, makeBounds(sstable, dataRange).iterator(), listener);
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        return getScanner(sstable, makeBounds(sstable, tokenRanges).iterator());
+    }
+
+    public static ISSTableScanner getScanner(BtiTableReader sstable, Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return new BtiTableScanner(sstable, ColumnFilter.all(sstable.metadata()), null, rangeIterator, SSTableReadsListener.NOOP_LISTENER);
+    }
+
+    private BtiTableScanner(BtiTableReader sstable,
+                            ColumnFilter columns,
+                            DataRange dataRange,
+                            Iterator<AbstractBounds<PartitionPosition>> rangeIterator,
+                            SSTableReadsListener listener)
+    {
+        assert sstable != null;
+
+        this.dfile = sstable.openDataReader();
+        this.sstable = sstable;
+        this.columns = columns;
+        this.dataRange = dataRange;
+        this.rangeIterator = rangeIterator;
+        this.listener = listener;
+    }
+
+    public static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, Collection<Range<Token>> tokenRanges)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(tokenRanges.size());
+        for (Range<Token> range : Range.normalize(tokenRanges))
+            addRange(sstable, Range.makeRowRange(range), boundsList);
+        return boundsList;
+    }
+
+    static List<AbstractBounds<PartitionPosition>> makeBounds(SSTableReader sstable, DataRange dataRange)
+    {
+        List<AbstractBounds<PartitionPosition>> boundsList = new ArrayList<>(2);
+        addRange(sstable, dataRange.keyRange(), boundsList);
+        return boundsList;
+    }
+
+    static AbstractBounds<PartitionPosition> fullRange(SSTableReader sstable)
+    {
+        return new Bounds<>(sstable.first, sstable.last);
+    }
+
+    private static void addRange(SSTableReader sstable, AbstractBounds<PartitionPosition> requested, List<AbstractBounds<PartitionPosition>> boundsList)
+    {
+        if (requested instanceof Range && ((Range<?>) requested).isWrapAround())
+        {
+            if (requested.right.compareTo(sstable.first) >= 0)
+            {
+                // since we wrap, we must contain the whole sstable prior to stopKey()
+                Boundary<PartitionPosition> left = new Boundary<>(sstable.first, true);
+                Boundary<PartitionPosition> right;
+                right = requested.rightBoundary();
+                right = minRight(right, sstable.last, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+            if (requested.left.compareTo(sstable.last) <= 0)
+            {
+                // since we wrap, we must contain the whole sstable after dataRange.startKey()
+                Boundary<PartitionPosition> right = new Boundary<>(sstable.last, true);
+                Boundary<PartitionPosition> left;
+                left = requested.leftBoundary();
+                left = maxLeft(left, sstable.first, true);
+                if (!isEmpty(left, right))
+                    boundsList.add(AbstractBounds.bounds(left, right));
+            }
+        }
+        else
+        {
+            assert !AbstractBounds.strictlyWrapsAround(requested.left, requested.right);
+            Boundary<PartitionPosition> left, right;
+            left = requested.leftBoundary();
+            right = requested.rightBoundary();
+            left = maxLeft(left, sstable.first, true);
+            // apparently isWrapAround() doesn't count Bounds that extend to the limit (min) as wrapping
+            right = requested.right.isMinimum() ? new Boundary<>(sstable.last, true)
+                                                : minRight(right, sstable.last, true);
+            if (!isEmpty(left, right))
+                boundsList.add(AbstractBounds.bounds(left, right));
+        }
+    }
+
+    public void close()
+    {
+        try
+        {
+            if (isClosed.compareAndSet(false, true))
+            {
+                FileUtils.close(dfile);
+                if (iterator != null)
+                    iterator.close();
+            }
+        }
+        catch (IOException e)
+        {
+            sstable.markSuspect();
+            throw new CorruptSSTableException(e, sstable.getFilename());
+        }
+    }
+
+    public long getBytesScanned()
+    {
+        return bytesScanned;
+    }
+
+    @Override
+    public long getLengthInBytes()
+    {
+        return sstable.uncompressedLength();
+    }
+
+
+    public long getCompressedLengthInBytes()
+    {
+        return sstable.onDiskLength();
+    }
+
+    @Override
+    public long getCurrentPosition()
+    {
+        return dfile.getFilePointer();
+    }
+
+    @Override
+    public Set<SSTableReader> getBackingSSTables()
+    {
+        return ImmutableSet.of(sstable);
+    }
+
+    public int level()

Review Comment:
   Removed.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *
+ * Not to be used outside of package. Public only for IndexRewriter tool.
+ */
+public class PartitionIndexBuilder implements AutoCloseable
+{
+    private final SequentialWriter writer;
+    private final IncrementalTrieWriter<PartitionIndex.Payload> trieWriter;
+    private final FileHandle.Builder fhBuilder;
+
+    // the last synced data file position
+    private long dataSyncPosition;
+    // the last synced row index file position
+    private long rowIndexSyncPosition;
+    // the last synced partition index file position
+    private long partitionIndexSyncPosition;
+
+    // Partial index can only be used after all three files have been synced to the required positions.
+    private long partialIndexDataEnd;
+    private long partialIndexRowEnd;
+    private long partialIndexPartitionEnd;
+    private IncrementalTrieWriter.PartialTail partialIndexTail;
+    private Consumer<PartitionIndex> partialIndexConsumer;
+    private DecoratedKey partialIndexLastKey;
+
+    private int lastDiffPoint;
+    private DecoratedKey firstKey;
+    private DecoratedKey lastKey;
+    private DecoratedKey lastWrittenKey;
+    private PartitionIndex.Payload lastPayload;
+
+    public PartitionIndexBuilder(SequentialWriter writer, FileHandle.Builder fhBuilder)
+    {
+        this.writer = writer;
+        this.trieWriter = IncrementalTrieWriter.open(PartitionIndex.TRIE_SERIALIZER, writer);
+        this.fhBuilder = fhBuilder;
+    }
+
+    /*
+     * Called when partition index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markPartitionIndexSynced(long upToPosition)
+    {
+        partitionIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when row index has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markRowIndexSynced(long upToPosition)
+    {
+        rowIndexSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    /*
+     * Called when data file has been flushed to the given position.
+     * If this makes all required positions for a partial view flushed, this will call the partialIndexConsumer.
+     */
+    public void markDataSynced(long upToPosition)
+    {
+        dataSyncPosition = upToPosition;
+        refreshReadableBoundary();
+    }
+
+    private void refreshReadableBoundary()
+    {
+        if (partialIndexConsumer == null)
+            return;
+        if (dataSyncPosition < partialIndexDataEnd)
+            return;
+        if (rowIndexSyncPosition < partialIndexRowEnd)
+            return;
+        if (partitionIndexSyncPosition < partialIndexPartitionEnd)
+            return;
+
+        try (FileHandle fh = fhBuilder.withLengthOverride(writer.getLastFlushOffset()).complete())
+        {
+            @SuppressWarnings("resource")
+            PartitionIndex pi = new PartitionIndexEarly(fh, partialIndexTail.root(), partialIndexTail.count(), firstKey, partialIndexLastKey, partialIndexTail.cutoff(), partialIndexTail.tail());
+            partialIndexConsumer.accept(pi);
+            partialIndexConsumer = null;
+        }
+        finally
+        {
+            fhBuilder.withLengthOverride(-1);
+        }
+
+    }
+
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexBuilder.class);

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *

Review Comment:
   Removed latter part (`IndexRewriter` does not exist) and made class package-private.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+
+                    if (online)
+                        FilterComponent.save(filter, descriptor, false);
+                }
+            }
+
+            if (builder.getFilter() == null)
+                builder.setFilter(FilterFactory.AlwaysPresent);
+
+            if (builder.getComponents().contains(Components.ROW_INDEX))
+                builder.setRowIndexFile(rowIndexFileBuilder().complete());
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX))
+            {
+                builder.setPartitionIndex(openPartitionIndex(builder.getFilter().isInformative()));
+                builder.setFirst(builder.getPartitionIndex().firstKey());
+                builder.setLast(builder.getPartitionIndex().lastKey());
+            }
+
+            try (CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components))
+            {
+                builder.setDataFile(dataFileBuilder(builder.getStatsMetadata()).withCompressionMetadata(compressionMetadata).complete());
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            // in case of failure, close only those components which have been opened in this try-catch block
+            Throwables.closeAndAddSuppressed(ex, builder.getPartitionIndex(), builder.getRowIndexFile(), builder.getDataFile(), builder.getFilter());
+            throw ex;
+        }
+    }
+
+    private IFilter buildBloomFilter(StatsMetadata statsMetadata) throws IOException
+    {
+        IFilter bf = null;
+
+        try (KeyReader keyReader = createKeyReader(statsMetadata))
+        {
+            if (keyReader == null)

Review Comment:
   Removed



##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,560 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableReader.class);
+
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);

Review Comment:
   Hmm. This seems to indicate a problem with the hierarchy -- it seems you can get different results depending on what type of reference you call `unbuildTo` on, as the methods don't override one another.
   
   @jacek-lewandowski, do you think there's a way to make this safer, or should we rename the methods to e.g. `unbuildBtiReaderTo` etc.?



##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;
+        IterationPosition prev;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;

Review Comment:
   Changed to `checkArgument` i.e. `InvalidArgumentException` which is as much as that message could say anyway.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())
+                    throw new CompactionInterruptedException(verifyInfo.getCompactionInfo());
+
+                rowStart = dataFile.getFilePointer();
+                outputHandler.debug("Reading row at %d", rowStart);
+
+                DecoratedKey key = null;
+                try
+                {
+                    key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
+                }
+                catch (Throwable th)
+                {
+                    throwIfFatal(th);

Review Comment:
   The verifier is not supposed to continue after any error -- changed to `markAndThrow`.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReverseIterator.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.PrintStream;
+
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.ReverseValueIterator;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reverse iterator over the row index. Needed to get previous index blocks for reverse iteration.
+ */
+class RowIndexReverseIterator extends ReverseValueIterator<RowIndexReverseIterator>
+{
+    private long currentNode = -1;
+
+    public RowIndexReverseIterator(FileHandle file, long root, ByteComparable start, ByteComparable end)
+    {
+        super(file.instantiateRebufferer(null), root, start, end, true);
+    }
+
+    public RowIndexReverseIterator(FileHandle file, TrieIndexEntry entry, ByteComparable end)
+    {
+        this(file, entry.indexTrieRoot, ByteComparable.EMPTY, end);
+    }
+
+    /**
+     * This method must be async-read-safe.
+     */
+    public IndexInfo nextIndexInfo()
+    {
+        if (currentNode == -1)
+        {
+            currentNode = nextPayloadedNode();
+            if (currentNode == -1)
+                return null;
+        }
+
+        go(currentNode);
+        IndexInfo info = RowIndexReader.readPayload(buf, payloadPosition(), payloadFlags());
+
+        currentNode = -1;
+        return info;
+    }
+
+    public void dumpTrie(PrintStream out)

Review Comment:
   Added `@SuppressWarnings`.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReverseIterator.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.PrintStream;
+
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.ReverseValueIterator;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reverse iterator over the row index. Needed to get previous index blocks for reverse iteration.
+ */
+class RowIndexReverseIterator extends ReverseValueIterator<RowIndexReverseIterator>
+{
+    private long currentNode = -1;
+
+    public RowIndexReverseIterator(FileHandle file, long root, ByteComparable start, ByteComparable end)
+    {
+        super(file.instantiateRebufferer(null), root, start, end, true);
+    }
+
+    public RowIndexReverseIterator(FileHandle file, TrieIndexEntry entry, ByteComparable end)
+    {
+        this(file, entry.indexTrieRoot, ByteComparable.EMPTY, end);
+    }
+
+    /**
+     * This method must be async-read-safe.
+     */
+    public IndexInfo nextIndexInfo()
+    {
+        if (currentNode == -1)
+        {
+            currentNode = nextPayloadedNode();
+            if (currentNode == -1)
+                return null;
+        }
+
+        go(currentNode);
+        IndexInfo info = RowIndexReader.readPayload(buf, payloadPosition(), payloadFlags());
+
+        currentNode = -1;
+        return info;
+    }
+
+    public void dumpTrie(PrintStream out)
+    {
+        dumpTrie(out, (buf, ppos, bits) -> {
+            IndexInfo ii = RowIndexReader.readPayload(buf, ppos, bits);

Review Comment:
   Extracted method in `RowIndexReader` to avoid code repetition.



##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         *
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;
+        IterationPosition prev;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/ValueIterator.java:
##########
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ValueIterator<CONCRETE extends ValueIterator<CONCRETE>> extends Walker<CONCRETE>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,398 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ */
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         *
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         *
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         *

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] Maxwell-Guo commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "Maxwell-Guo (via GitHub)" <gi...@apache.org>.
Maxwell-Guo commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1503035868

   One more question , is it possible that one node have two sstable format (BIG and BIT)under one table ? if it is possible , and how is the compact done? BIT and BIG doing compaction for their own ? or mixed together ? and I think if it is possible ,we should add some compaction test case for this 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1162672838


##########
conf/cassandra.yaml:
##########
@@ -1924,8 +1924,14 @@ drop_compact_storage_enabled: false
 # which is used in some serialization to denote the format type in a compact way (such as local key cache); and 'name'
 # which will be used to recognize the format type - in particular that name will be used in sstable file names and in
 # stream headers so the name has to be the same for the same format across all the nodes in the cluster.
+# The first entry in this list is the format that will be used for newly-created SSTables. The other formats given
+# will be used to read any SSTables present in the data directories or streamed.
 sstable_formats:
   - class_name: org.apache.cassandra.io.sstable.format.big.BigFormat
     parameters:
       id: 0
       name: big
+  - class_name: org.apache.cassandra.io.sstable.format.bti.BtiFormat
+    parameters:
+      id: 1
+      name: bti

Review Comment:
   Let us continue this discussion in CASSANDRA-18441.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180073926


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ * <p>
+ * Uses IncrementalTrieWriter to build a trie of index section separators of the shortest possible length such that
+ * prevMax < separator <= nextMin.
+ */
+class RowIndexWriter implements AutoCloseable
+{
+    private final ClusteringComparator comparator;
+    private final IncrementalTrieWriter<IndexInfo> trie;
+    private ByteComparable prevMax = null;
+    private ByteComparable prevSep = null;
+
+    RowIndexWriter(ClusteringComparator comparator, DataOutputPlus out)
+    {
+        this.comparator = comparator;
+        this.trie = IncrementalTrieWriter.open(RowIndexReader.trieSerializer, out);
+    }
+
+    void reset()
+    {
+        prevMax = null;
+        prevSep = null;
+        trie.reset();
+    }
+
+    @Override
+    public void close()
+    {
+        trie.close();
+    }
+
+    void add(ClusteringPrefix<?> firstName, ClusteringPrefix<?> lastName, IndexInfo info) throws IOException
+    {
+        assert info.openDeletion != null;
+        ByteComparable sep = prevMax == null
+                             ? ByteComparable.EMPTY
+                             : ByteComparable.separatorGt(prevMax, comparator.asByteComparable(firstName));
+        trie.add(sep, info);
+        prevSep = sep;
+        prevMax = comparator.asByteComparable(lastName);
+    }
+
+    public long complete(long endPos) throws IOException
+    {
+        // Add a separator after the last section, so that greater inputs can be quickly rejected.
+        // To maximize its efficiency we add it with the length of the last added separator.
+        int i = 0;
+        ByteSource max = prevMax.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        ByteSource sep = prevSep.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        int c;
+        while ((c = max.next()) == sep.next() && c != ByteSource.END_OF_STREAM)
+            ++i;
+        assert c != ByteSource.END_OF_STREAM : "Corrupted row order, max=" + prevMax;
+
+        trie.add(nudge(prevMax, i), new IndexInfo(endPos, DeletionTime.LIVE));
+
+        return trie.complete();
+    }
+
+    /**
+     * Produces a source that is slightly greater than argument with length at least nudgeAt.
+     */
+    private ByteComparable nudge(ByteComparable value, int nudgeAt)
+    {
+        return version -> new ByteSource()
+        {
+            private final ByteSource v = value.asComparableBytes(version);
+            private int cur = 0;
+
+            @Override
+            public int next()
+            {
+                int b = ByteSource.END_OF_STREAM;
+                if (cur <= nudgeAt)
+                {
+                    b = v.next();
+                    if (cur == nudgeAt)
+                    {
+                        if (b < 255)

Review Comment:
   I personally prefer not to hide fundamental constants behind identifiers as they can mislead and hide errors.
   
   Changed to `0xFF` to make it even more obvious.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)

Review Comment:
   Added (also to `TrieNode`).



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            IndexInfo payload = node.payload();
+            int bytes = 0;
+            int hasOpenMarker = 0;
+            if (payload != null)
+            {
+                bytes = SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    hasOpenMarker = FLAG_OPEN_MARKER;
+            }
+            type.serialize(dest, node, bytes | hasOpenMarker, nodePosition);

Review Comment:
   It is, if the offset is over 32PiB.
   
   Added an assertion to fail with a sensible message if that should ever happen.



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ * <p>
+ * Uses IncrementalTrieWriter to build a trie of index section separators of the shortest possible length such that

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndex.java:
##########
@@ -0,0 +1,445 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.ValueIterator;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.concurrent.Ref;
+import org.apache.cassandra.utils.concurrent.SharedCloseable;
+
+/**
+ * This class holds the partition index as an on-disk trie mapping unique prefixes of decorated keys to:
+ *     - data file position if the partition is small enough to not need an index
+ *     - row index file position if the partition has a row index
+ * plus
+ *     - the last 8 bits of the key's filter hash which is used to filter out mismatched keys without reading the key
+ *
+ * To avoid having to create an object to carry the result, the two are distinguished by sign. Direct-to-dfile entries
+ * are recorded as ~position (~ instead of - to differentiate 0 in ifile from 0 in dfile).
+ *
+ * In either case the contents of the file at this position start with a serialization of the key which can be used
+ * to verify the correct key is found.
+ *
+ * To read the index one must obtain a thread-unsafe Reader or IndexPosIterator.
+ */
+@VisibleForTesting
+public class PartitionIndex implements SharedCloseable
+{
+    private static final Logger logger = LoggerFactory.getLogger(PartitionIndex.class);
+
+    private final FileHandle fh;
+    private final long keyCount;
+    private final DecoratedKey first;
+    private final DecoratedKey last;
+    private final long root;
+
+    public static final long NOT_FOUND = Long.MIN_VALUE;
+    public static final int FOOTER_LENGTH = 3 * 8;
+
+    @VisibleForTesting
+    public PartitionIndex(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last)
+    {
+        this.keyCount = keyCount;
+        this.fh = fh.sharedCopy();
+        this.first = first;
+        this.last = last;
+        this.root = trieRoot;
+    }
+
+    private PartitionIndex(PartitionIndex src)
+    {
+        this(src.fh, src.root, src.keyCount, src.first, src.last);
+    }
+
+    static class Payload
+    {
+        final long position;
+        final short hashBits;
+
+        public Payload(long position, short hashBits)
+        {
+            this.position = position;
+            assert this.position != NOT_FOUND;
+            this.hashBits = hashBits;
+        }
+    }
+
+    static final PartitionIndexSerializer TRIE_SERIALIZER = new PartitionIndexSerializer();
+
+    private static class PartitionIndexSerializer implements TrieSerializer<Payload, DataOutputPlus>
+    {
+        public int sizeofNode(SerializationNode<Payload> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) +
+                   (node.payload() != null ? 1 + SizedInts.nonZeroSize(node.payload().position) : 0);
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<Payload> node, long nodePosition) throws IOException
+        {
+            Payload payload = node.payload();
+            if (payload != null)
+            {
+                int payloadBits;
+                int size = SizedInts.nonZeroSize(payload.position);
+                payloadBits = 7 + size;
+                type.serialize(dest, node, payloadBits, nodePosition);
+                dest.writeByte(payload.hashBits);
+                SizedInts.write(dest, payload.position, size);
+            }
+            else
+                type.serialize(dest, node, 0, nodePosition);
+        }
+    }
+
+    public long size()
+    {
+        return keyCount;
+    }
+
+    public DecoratedKey firstKey()
+    {
+        return first;
+    }
+
+    public DecoratedKey lastKey()
+    {
+        return last;
+    }
+
+    @Override
+    public PartitionIndex sharedCopy()
+    {
+        return new PartitionIndex(this);
+    }
+
+    @Override
+    public void addTo(Ref.IdentityCollection identities)
+    {
+        fh.addTo(identities);
+    }
+
+    public static PartitionIndex load(FileHandle.Builder fhBuilder,
+                                      IPartitioner partitioner,
+                                      boolean preload) throws IOException
+    {
+        try (FileHandle fh = fhBuilder.complete())
+        {
+            return load(fh, partitioner, preload);
+        }
+    }
+
+    public static Pair<DecoratedKey, DecoratedKey> readFirstAndLastKey(File file, IPartitioner partitioner) throws IOException
+    {
+        try (PartitionIndex index = load(new FileHandle.Builder(file), partitioner, false))
+        {
+            return Pair.create(index.firstKey(), index.lastKey());
+        }
+    }
+
+    public static PartitionIndex load(FileHandle fh, IPartitioner partitioner, boolean preload) throws IOException
+    {
+        try (FileDataInput rdr = fh.createReader(fh.dataLength() - FOOTER_LENGTH))
+        {
+            long firstPos = rdr.readLong();
+            long keyCount = rdr.readLong();
+            long root = rdr.readLong();
+            rdr.seek(firstPos);
+            DecoratedKey first = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            DecoratedKey last = partitioner != null ? partitioner.decorateKey(ByteBufferUtil.readWithShortLength(rdr)) : null;
+            if (preload)
+            {
+                int csum = 0;
+                // force a read of all the pages of the index
+                for (long pos = 0; pos < fh.dataLength(); pos += PageAware.PAGE_SIZE)
+                {
+                    rdr.seek(pos);
+                    csum += rdr.readByte();
+                }
+                logger.trace("Checksum {}", csum);      // Note: trace is required so that reads aren't optimized away.
+            }
+
+            return new PartitionIndex(fh, root, keyCount, first, last);
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        fh.close();
+    }
+
+    @Override
+    public Throwable close(Throwable accumulate)
+    {
+        return fh.close(accumulate);
+    }
+
+    public Reader openReader()
+    {
+        return new Reader(this);
+    }
+
+    protected IndexPosIterator allKeysIterator()
+    {
+        return new IndexPosIterator(this);
+    }
+
+    protected Rebufferer instantiateRebufferer()
+    {
+        return fh.instantiateRebufferer(null);
+    }
+
+
+    /**
+     * @return the file handle to the file on disk. This is needed for locking the index in RAM.
+     */
+    FileHandle getFileHandle()
+    {
+        return fh;
+    }
+
+    private static long getIndexPos(ByteBuffer contents, int payloadPos, int bytes)
+    {
+        if (bytes > 7)
+        {
+            ++payloadPos;
+            bytes -= 7;
+        }
+        if (bytes == 0)
+            return NOT_FOUND;
+        return SizedInts.read(contents, payloadPos, bytes);
+    }
+
+    public interface Acceptor<ArgType, ResultType>
+    {
+        ResultType accept(long position, boolean assumeNoMatch, ArgType v) throws IOException;
+    }
+
+    /**
+     * Provides methods to read the partition index trie.
+     * Thread-unsafe, uses class members to store lookup state.
+     */
+    public static class Reader extends Walker<Reader>

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            IndexInfo payload = node.payload();
+            int bytes = 0;
+            int hasOpenMarker = 0;
+            if (payload != null)
+            {
+                bytes = SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    hasOpenMarker = FLAG_OPEN_MARKER;
+            }
+            type.serialize(dest, node, bytes | hasOpenMarker, nodePosition);
+            if (payload != null)
+            {
+                SizedInts.write(dest, payload.offset, bytes);
+
+                if (hasOpenMarker != 0)

Review Comment:
   Changed



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    protected TransactionalProxy txnProxy()
+    {
+        return new TransactionalProxy(() -> FBUtilities.immutableListWithFilteredNulls(iwriter, dataWriter));
+    }
+
+    private class TransactionalProxy extends SortedTableWriter<BtiFormatPartitionWriter>.TransactionalProxy
+    {
+        public TransactionalProxy(Supplier<ImmutableList<Transactional>> transactionals)
+        {
+            super(transactionals);
+        }
+
+        @Override
+        protected Throwable doPostCleanup(Throwable accumulate)
+        {
+            accumulate = Throwables.close(accumulate, partitionWriter);
+            accumulate = super.doPostCleanup(accumulate);
+            return accumulate;
+        }
+    }
+
+    /**
+     * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed.
+     */
+    static class IndexWriter extends SortedTableWriter.AbstractIndexWriter
+    {
+        final SequentialWriter rowIndexWriter;
+        private final FileHandle.Builder rowIndexFHBuilder;
+        private final SequentialWriter partitionIndexWriter;
+        private final FileHandle.Builder partitionIndexFHBuilder;
+        private final PartitionIndexBuilder partitionIndex;
+        boolean partitionIndexCompleted = false;
+        private DataPosition riMark;
+        private DataPosition piMark;
+
+        IndexWriter(Builder b)
+        {
+            super(b);
+            rowIndexWriter = new SequentialWriter(descriptor.fileFor(Components.ROW_INDEX), b.getIOOptions().writerOptions);
+            rowIndexFHBuilder = IndexComponent.fileBuilder(Components.ROW_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndexWriter = new SequentialWriter(descriptor.fileFor(Components.PARTITION_INDEX), b.getIOOptions().writerOptions);
+            partitionIndexFHBuilder = IndexComponent.fileBuilder(Components.PARTITION_INDEX, b).withMmappedRegionsCache(b.getMmappedRegionsCache());
+            partitionIndex = new PartitionIndexBuilder(partitionIndexWriter, partitionIndexFHBuilder);
+            // register listeners to be alerted when the data files are flushed
+            partitionIndexWriter.setPostFlushListener(() -> partitionIndex.markPartitionIndexSynced(partitionIndexWriter.getLastFlushOffset()));
+            rowIndexWriter.setPostFlushListener(() -> partitionIndex.markRowIndexSynced(rowIndexWriter.getLastFlushOffset()));
+            @SuppressWarnings("resource")

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()
+    {
+        // we must ensure the data is completely flushed to disk
+        iwriter.complete(); // This will be called by completedPartitionIndex() below too, but we want it done now to
+        // ensure outstanding openEarly actions are not triggered.
+        dataWriter.sync();
+        iwriter.rowIndexWriter.sync();
+        // Note: Nothing must be written to any of the files after this point, as the chunk cache could pick up and
+        // retain a partially-written page (see DB-2446).
+
+        return openFinal(OpenReason.EARLY);
+    }
+
+    @SuppressWarnings("resource")
+    protected SSTableReader openFinal(OpenReason openReason)
+    {
+
+        if (maxDataAge < 0)
+            maxDataAge = Clock.Global.currentTimeMillis();
+
+        return openInternal(openReason, true, iwriter::completedPartitionIndex);
+    }
+
+    protected TransactionalProxy txnProxy()

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)
+    {
+        long dataLength = dataWriter.position();
+        iwriter.buildPartial(dataLength, partitionIndex ->
+        {
+            iwriter.rowIndexFHBuilder.withLengthOverride(iwriter.rowIndexWriter.getLastFlushOffset());
+            BtiTableReader reader = openInternal(OpenReason.EARLY, false, () -> partitionIndex);
+            callWhenReady.accept(reader);
+        });
+    }
+
+    public SSTableReader openFinalEarly()

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180563816


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexWriter.java:
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Preparer / writer of row index tries.
+ * <p>
+ * Uses IncrementalTrieWriter to build a trie of index section separators of the shortest possible length such that
+ * prevMax < separator <= nextMin.
+ */
+class RowIndexWriter implements AutoCloseable
+{
+    private final ClusteringComparator comparator;
+    private final IncrementalTrieWriter<IndexInfo> trie;
+    private ByteComparable prevMax = null;
+    private ByteComparable prevSep = null;
+
+    RowIndexWriter(ClusteringComparator comparator, DataOutputPlus out)
+    {
+        this.comparator = comparator;
+        this.trie = IncrementalTrieWriter.open(RowIndexReader.trieSerializer, out);
+    }
+
+    void reset()
+    {
+        prevMax = null;
+        prevSep = null;
+        trie.reset();
+    }
+
+    @Override
+    public void close()
+    {
+        trie.close();
+    }
+
+    void add(ClusteringPrefix<?> firstName, ClusteringPrefix<?> lastName, IndexInfo info) throws IOException
+    {
+        assert info.openDeletion != null;
+        ByteComparable sep = prevMax == null
+                             ? ByteComparable.EMPTY
+                             : ByteComparable.separatorGt(prevMax, comparator.asByteComparable(firstName));
+        trie.add(sep, info);
+        prevSep = sep;
+        prevMax = comparator.asByteComparable(lastName);
+    }
+
+    public long complete(long endPos) throws IOException
+    {
+        // Add a separator after the last section, so that greater inputs can be quickly rejected.
+        // To maximize its efficiency we add it with the length of the last added separator.
+        int i = 0;
+        ByteSource max = prevMax.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        ByteSource sep = prevSep.asComparableBytes(Walker.BYTE_COMPARABLE_VERSION);
+        int c;
+        while ((c = max.next()) == sep.next() && c != ByteSource.END_OF_STREAM)
+            ++i;
+        assert c != ByteSource.END_OF_STREAM : "Corrupted row order, max=" + prevMax;
+
+        trie.add(nudge(prevMax, i), new IndexInfo(endPos, DeletionTime.LIVE));
+
+        return trie.complete();
+    }
+
+    /**
+     * Produces a source that is slightly greater than argument with length at least nudgeAt.
+     */
+    private ByteComparable nudge(ByteComparable value, int nudgeAt)
+    {
+        return version -> new ByteSource()
+        {
+            private final ByteSource v = value.asComparableBytes(version);
+            private int cur = 0;
+
+            @Override
+            public int next()
+            {
+                int b = ByteSource.END_OF_STREAM;
+                if (cur <= nudgeAt)
+                {
+                    b = v.next();
+                    if (cur == nudgeAt)
+                    {
+                        if (b < 255)

Review Comment:
   Works for me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179749211


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableWriter.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
+import org.apache.cassandra.io.FSReadError;
+import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.DataComponent;
+import org.apache.cassandra.io.sstable.format.IndexComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.OpenReason;
+import org.apache.cassandra.io.sstable.format.SortedTableWriter;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.DataPosition;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.MmappedRegionsCache;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Clock;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static org.apache.cassandra.io.util.FileHandle.Builder.NO_LENGTH_OVERRIDE;
+
+@VisibleForTesting
+public class BtiTableWriter extends SortedTableWriter<BtiFormatPartitionWriter>
+{
+    private static final Logger logger = LoggerFactory.getLogger(BtiTableWriter.class);
+
+    private final BtiFormatPartitionWriter partitionWriter;
+    private final IndexWriter iwriter;
+
+    public BtiTableWriter(Builder builder, LifecycleNewTracker lifecycleNewTracker, SSTable.Owner owner)
+    {
+        super(builder, lifecycleNewTracker, owner);
+        this.iwriter = builder.getIndexWriter();
+        this.partitionWriter = builder.getPartitionWriter();
+    }
+
+    @Override
+    public void mark()
+    {
+        super.mark();
+        iwriter.mark();
+    }
+
+    @Override
+    public void resetAndTruncate()
+    {
+        super.resetAndTruncate();
+        iwriter.resetAndTruncate();
+    }
+
+    @Override
+    protected TrieIndexEntry createRowIndexEntry(DecoratedKey key, DeletionTime partitionLevelDeletion, long finishResult) throws IOException
+    {
+        TrieIndexEntry entry = TrieIndexEntry.create(partitionWriter.getInitialPosition(),
+                                                     finishResult,
+                                                     partitionLevelDeletion,
+                                                     partitionWriter.getRowIndexCount());
+        iwriter.append(key, entry);
+        return entry;
+    }
+
+    @SuppressWarnings("resource")
+    private BtiTableReader openInternal(OpenReason openReason, boolean isFinal, Supplier<PartitionIndex> partitionIndexSupplier)
+    {
+        IFilter filter = null;
+        FileHandle dataFile = null;
+        PartitionIndex partitionIndex = null;
+        FileHandle rowIndexFile = null;
+
+        BtiTableReader.Builder builder = unbuildTo(new BtiTableReader.Builder(descriptor), true).setMaxDataAge(maxDataAge)
+                                                                                                .setSerializationHeader(header)
+                                                                                                .setOpenReason(openReason);
+
+        try
+        {
+            builder.setStatsMetadata(statsMetadata());
+
+            partitionIndex = partitionIndexSupplier.get();
+            rowIndexFile = iwriter.rowIndexFHBuilder.complete();
+            dataFile = openDataFile(isFinal ? NO_LENGTH_OVERRIDE : dataWriter.getLastFlushOffset(), builder.getStatsMetadata());
+            filter = iwriter.getFilterCopy();
+
+            return builder.setPartitionIndex(partitionIndex)
+                          .setFirst(partitionIndex.firstKey())
+                          .setLast(partitionIndex.lastKey())
+                          .setRowIndexFile(rowIndexFile)
+                          .setDataFile(dataFile)
+                          .setFilter(filter)
+                          .build(owner().orElse(null), true, true);
+        }
+        catch (RuntimeException | Error ex)
+        {
+            JVMStabilityInspector.inspectThrowable(ex);
+            Throwables.closeNonNullAndAddSuppressed(ex, filter, dataFile, rowIndexFile, partitionIndex);
+            throw ex;
+        }
+    }
+
+
+    public void openEarly(Consumer<SSTableReader> callWhenReady)

Review Comment:
   nit: `@Override`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1179695424


##########
src/java/org/apache/cassandra/io/sstable/format/bti/RowIndexReader.java:
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.tries.SerializationNode;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.TrieSerializer;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SizedInts;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Reader class for row index files.
+ *
+ * Row index "tries" do not need to store whole keys, as what we need from them is to be able to tell where in the data file
+ * to start looking for a given key. Instead, we store some prefix that is greater than the greatest key of the previous
+ * index section and smaller than or equal to the smallest key of the next. So for a given key the first index section
+ * that could potentially contain it is given by the trie's floor for that key.
+ *
+ * This builds upon the trie Walker class which provides basic trie walking functionality. The class is thread-unsafe
+ * and must be re-instantiated for every thread that needs access to the trie (its overhead is below that of a
+ * RandomAccessReader).
+ */
+public class RowIndexReader extends Walker<RowIndexReader>
+{
+    private static final int FLAG_OPEN_MARKER = 8;
+
+    public static class IndexInfo
+    {
+        public final long offset;
+        public final DeletionTime openDeletion;
+
+        IndexInfo(long offset, DeletionTime openDeletion)
+        {
+            this.offset = offset;
+            this.openDeletion = openDeletion;
+        }
+    }
+
+    public RowIndexReader(FileHandle file, long root)
+    {
+        super(file.instantiateRebufferer(null), root);
+    }
+
+    public RowIndexReader(FileHandle file, TrieIndexEntry entry)
+    {
+        this(file, entry.indexTrieRoot);
+    }
+
+    /**
+     * Computes the floor for a given key.
+     */
+    public IndexInfo separatorFloor(ByteComparable key)
+    {
+        // Check for a prefix and find closest smaller branch.
+        IndexInfo res = prefixAndNeighbours(key, RowIndexReader::readPayload);
+        // If there's a prefix, in a separator trie it could be less than, equal, or greater than sought value.
+        // Sought value is still greater than max of previous section.
+        // On match the prefix must be used as a starting point.
+        if (res != null)
+            return res;
+
+        // Otherwise return the IndexInfo for the closest entry of the smaller branch (which is the max of lesserBranch).
+        // Note (see prefixAndNeighbours): since we accept prefix matches above, at this point there cannot be another
+        // prefix match that is closer than max(lesserBranch).
+        if (lesserBranch == -1)
+            return null;
+        goMax(lesserBranch);
+        return getCurrentIndexInfo();
+    }
+
+    public IndexInfo min()
+    {
+        goMin(root);
+        return getCurrentIndexInfo();
+    }
+
+    protected IndexInfo getCurrentIndexInfo()
+    {
+        return readPayload(payloadPosition(), payloadFlags());
+    }
+
+    protected IndexInfo readPayload(int ppos, int bits)
+    {
+        return readPayload(buf, ppos, bits);
+    }
+
+    static IndexInfo readPayload(ByteBuffer buf, int ppos, int bits)
+    {
+        long dataOffset;
+        if (bits == 0)
+            return null;
+        int bytes = bits & ~FLAG_OPEN_MARKER;
+        dataOffset = SizedInts.read(buf, ppos, bytes);
+        ppos += bytes;
+        DeletionTime deletion = (bits & FLAG_OPEN_MARKER) != 0
+                ? DeletionTime.serializer.deserialize(buf, ppos)
+                : null;
+        return new IndexInfo(dataOffset, deletion);
+    }
+
+    // The trie serializer describes how the payloads are written. Placed here (instead of writer) so that reading and
+    // writing the payload are close together should they need to be changed.
+    static final TrieSerializer<IndexInfo, DataOutputPlus> trieSerializer = new TrieSerializer<IndexInfo, DataOutputPlus>()
+    {
+        @Override
+        public int sizeofNode(SerializationNode<IndexInfo> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + sizeof(node.payload());
+        }
+
+        @Override
+        public void write(DataOutputPlus dest, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            write(dest, TrieNode.typeFor(node, nodePosition), node, nodePosition);
+        }
+
+        public int sizeof(IndexInfo payload)
+        {
+            int size = 0;
+            if (payload != null)
+            {
+                size += SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    size += DeletionTime.serializer.serializedSize(payload.openDeletion);
+            }
+            return size;
+        }
+
+        public void write(DataOutputPlus dest, TrieNode type, SerializationNode<IndexInfo> node, long nodePosition) throws IOException
+        {
+            IndexInfo payload = node.payload();
+            int bytes = 0;
+            int hasOpenMarker = 0;
+            if (payload != null)
+            {
+                bytes = SizedInts.nonZeroSize(payload.offset);
+                if (!payload.openDeletion.isLive())
+                    hasOpenMarker = FLAG_OPEN_MARKER;
+            }
+            type.serialize(dest, node, bytes | hasOpenMarker, nodePosition);

Review Comment:
   Just to confirm, it's not actually possible for `bytes` to be `8` and interfere w/ `hasOpenMarker`, correct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1180813975


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1178645784


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormatPartitionWriter.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.format.SortedTablePartitionWriter;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.SequentialWriter;
+
+/**
+ * Partition writer used by {@link BtiTableWriter}.
+ * <p>
+ * Writes all passed data to the given SequentialWriter and if necessary builds a RowIndex by constructing an entry
+ * for each row within a partition that follows {@link org.apache.cassandra.config.Config#column_index_size} of written
+ * data.
+ */
+class BtiFormatPartitionWriter extends SortedTablePartitionWriter
+{
+    private final RowIndexWriter rowTrie;
+    private final int indexSize;
+    private int rowIndexCount;

Review Comment:
   nit: I'd consider naming this `blockCount`, as it's the number of blocks already written, correct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1170686863


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiFormat.md:
##########
@@ -0,0 +1,991 @@
+Big Trie-Indexed (BTI) SSTable format
+-------------------------------------
+
+This document describes the BTI SSTable format, which is introduced to
+Cassandra with [CEP-25](https://cwiki.apache.org/confluence/display/CASSANDRA/CEP-25%3A+Trie-indexed+SSTable+format).
+
+The format is called BTI, which stands for "Big Trie-Indexed", because it shares
+the data format of the existing BIG format and only changes the primary indexes
+inside SSTables. The format relies on byte order and tries, and uses a combination
+of features to make the indexing structures compact and efficient. The paragraphs
+below describe the format's features and mechanisms together with the motivation 
+behind them, and conclude with detailed description of the on-disk format.
+
+# Prerequisites
+
+## Byte-comparable types
+
+The property of being byte-comparable (also called byte-ordered) for a
+key denotes that there is a serialisation of that key to a sequence of
+bytes where the lexicographic comparison of the unsigned bytes produces
+the same result as performing a typed comparison of the key.
+
+For Cassandra, such a representation is given by
+[CASSANDRA-6936](https://issues.apache.org/jira/browse/CASSANDRA-6936).
+Detailed description of the mechanics of the translation are provided in
+the [included documentation](../../../../utils/bytecomparable/ByteComparable.md).
+
+## Tries
+
+A trie is a data structure that describes a mapping between sequences
+and associated values. It is very similar to a deterministic finite
+state automaton, with the main difference that an automaton is allowed
+to have cycles, while a trie is not.
+
+Because the theory and main usage of the structure is for encoding words
+of a language, the trie terminology talks about "characters", "words"
+and "alphabet", which in our case map to bytes of the byte-ordered
+representation, the sequence that encodes it, and the possible values of
+a byte[^1].
+
+[^1]: For simplicity this description assumes we directly map a byte to
+a character. Other options are also possible (e.g. using hex digits
+as the alphabet and two transitions per byte).
+
+A trie can be defined as a tree graph in which vertices are states, some
+of which can be final and contain associated information, and where
+edges are labelled with characters. A valid word in the trie is encoded
+by a path starting from the root of the trie where each edge is labelled
+with the next character of the word, and ending in a final state which
+contains the 'payload' associated with the word.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+    Node_a --"l"--> Node_al(( ))
+      Node_al --"l"--> Node_all(( ))
+        Node_all --"o"--> Node_allo(( ))
+          Node_allo --"w"--> Node_allow(((allow)))
+    Node_a --"n"--> Node_an(((an)))
+      Node_an --"d"--> Node_and(((and)))
+      Node_an --"y"--> Node_any(((any)))
+    Node_a --"r"--> Node_ar(( ))
+      Node_ar --"e"--> Node_are(((are)))
+    Node_a --"s"--> Node_as(((as)))
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+    Node_t --"o"--> Node_to(((to)))
+    Node_t --"r"--> Node_tr(( ))
+      Node_tr --"i"--> Node_tri(( ))
+        Node_tri --"e"--> Node_trie(((trie)))
+    Node_t --"y"--> Node_ty(( ))
+      Node_ty --"p"--> Node_typ(( ))
+        Node_typ --"e"--> Node_type(( ))
+          Node_type --"s"--> Node_types(((types)))
+  Node_ --"w"--> Node_w(( ))
+    Node_w --"i"--> Node_wi(( ))
+      Node_wi --"t"--> Node_wit(( ))
+        Node_wit --"h"--> Node_with(((with)))
+          Node_with --"o"--> Node_witho(( ))
+            Node_witho --"u"--> Node_withou(( ))
+              Node_withou --"t"--> Node_without(((without)))
+```
+
+This means that in a constructed trie finding the payload associated
+with a word is a matter of following the edges (also called
+"transitions") from the initial state labelled with the consecutive
+characters of the word, and retrieving the payload associated with the
+state at which we end up. If that's not a final state, or if at any
+point in this we did not find a transition in the trie matching the
+character, the trie does not have an association for the word. The
+complexity of lookup is thus _O_(len(word)) transitions, where the cost of
+taking a transition is usually constant, thus this complexity is
+theoretically optimal.
+
+From a storage space perspective, one of the main benefits of a trie as
+a data structure for storing a map is the fact that it completely avoids
+storing redundant prefixes. All words that start with the same sequence
+store a representation of that sequence only once. If prefixes are
+commonly shared, this can save a great deal of space.
+
+When the items stored in a trie are lexicographically (=byte) ordered, a
+trie is also an ordered structure. A trie can be walked in order and it
+is also possible to efficiently list the items between two given keys.
+
+In fact, one can efficiently (and lazily) apply set algebra over tries,
+and slicing can be seen as a simple application of intersection, where
+the intersecting trie is generated on the fly. The set operations
+benefit from the same prefix-sharing effect &mdash; we apply union /
+intersection / difference to a state, which has the effect of applying
+the operation to all words that share the prefix denoted by that state.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_ --"a"--> Node_a(((a)))
+  style Node_a stroke:lightgrey,color:lightgrey
+  linkStyle 0 stroke:lightgrey,color:lightgrey
+    Node_a --"l"--> Node_al(( ))
+    style Node_al stroke:lightgrey,color:lightgrey
+    linkStyle 1 stroke:lightgrey,color:lightgrey
+      Node_al --"l"--> Node_all(( ))
+      style Node_all stroke:lightgrey,color:lightgrey
+      linkStyle 2 stroke:lightgrey,color:lightgrey
+        Node_all --"o"--> Node_allo(( ))
+        style Node_allo stroke:lightgrey,color:lightgrey
+        linkStyle 3 stroke:lightgrey,color:lightgrey
+          Node_allo --"w"--> Node_allow(((allow)))
+          style Node_allow stroke:lightgrey,color:lightgrey
+          linkStyle 4 stroke:lightgrey,color:lightgrey
+    Node_a --"n"--> Node_an(((an)))
+          style Node_an stroke:lightgrey,color:lightgrey
+          linkStyle 5 stroke:lightgrey,color:lightgrey
+      Node_an --"d"--> Node_and(((and)))
+          style Node_and stroke:lightgrey,color:lightgrey
+          linkStyle 6 stroke:lightgrey,color:lightgrey
+      Node_an --"y"--> Node_any(((any)))
+          style Node_any stroke:lightgrey,color:lightgrey
+          linkStyle 7 stroke:lightgrey,color:lightgrey
+    Node_a --"r"--> Node_ar(( ))
+          style Node_ar stroke:lightgrey,color:lightgrey
+          linkStyle 8 stroke:lightgrey,color:lightgrey
+      Node_ar --"e"--> Node_are(((are)))
+          style Node_are stroke:lightgrey,color:lightgrey
+          linkStyle 9 stroke:lightgrey,color:lightgrey
+    Node_a --"s"--> Node_as(((as)))
+          style Node_as stroke:lightgrey,color:lightgrey
+          linkStyle 10 stroke:lightgrey,color:lightgrey
+  Node_ --"n"--> Node_n(( ))
+    Node_n --"o"--> Node_no(( ))
+      Node_no --"d"--> Node_nod(( ))
+        Node_nod --"e"--> Node_node(((node)))
+  Node_ --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of(((of)))
+    Node_o --"n"--> Node_on(((on)))
+  Node_ --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the(((the)))
+      Node_th --"i"--> Node_thi(( ))
+        Node_thi --"s"--> Node_this(((this)))
+          style Node_this stroke:lightgrey,color:lightgrey
+          linkStyle 22 stroke:lightgrey,color:lightgrey
+    Node_t --"o"--> Node_to(((to)))
+          style Node_to stroke:lightgrey,color:lightgrey
+          linkStyle 23 stroke:lightgrey,color:lightgrey
+    Node_t --"r"--> Node_tr(( ))
+          style Node_tr stroke:lightgrey,color:lightgrey
+          linkStyle 24 stroke:lightgrey,color:lightgrey
+      Node_tr --"i"--> Node_tri(( ))
+          style Node_tri stroke:lightgrey,color:lightgrey
+          linkStyle 25 stroke:lightgrey,color:lightgrey
+        Node_tri --"e"--> Node_trie(((trie)))
+          style Node_trie stroke:lightgrey,color:lightgrey
+          linkStyle 26 stroke:lightgrey,color:lightgrey
+    Node_t --"y"--> Node_ty(( ))
+          style Node_ty stroke:lightgrey,color:lightgrey
+          linkStyle 27 stroke:lightgrey,color:lightgrey
+      Node_ty --"p"--> Node_typ(( ))
+          style Node_typ stroke:lightgrey,color:lightgrey
+          linkStyle 28 stroke:lightgrey,color:lightgrey
+        Node_typ --"e"--> Node_type(( ))
+          style Node_type stroke:lightgrey,color:lightgrey
+          linkStyle 29 stroke:lightgrey,color:lightgrey
+          Node_type --"s"--> Node_types(((types)))
+          style Node_types stroke:lightgrey,color:lightgrey
+          linkStyle 30 stroke:lightgrey,color:lightgrey
+  Node_ --"w"--> Node_w(( ))
+          style Node_w stroke:lightgrey,color:lightgrey
+          linkStyle 31 stroke:lightgrey,color:lightgrey
+    Node_w --"i"--> Node_wi(( ))
+          style Node_wi stroke:lightgrey,color:lightgrey
+          linkStyle 32 stroke:lightgrey,color:lightgrey
+      Node_wi --"t"--> Node_wit(( ))
+          style Node_wit stroke:lightgrey,color:lightgrey
+          linkStyle 33 stroke:lightgrey,color:lightgrey
+        Node_wit --"h"--> Node_with(((with)))
+          style Node_with stroke:lightgrey,color:lightgrey
+          linkStyle 34 stroke:lightgrey,color:lightgrey
+          Node_with --"o"--> Node_witho(( ))
+          style Node_witho stroke:lightgrey,color:lightgrey
+          linkStyle 35 stroke:lightgrey,color:lightgrey
+            Node_witho --"u"--> Node_withou(( ))
+          style Node_withou stroke:lightgrey,color:lightgrey
+          linkStyle 36 stroke:lightgrey,color:lightgrey
+              Node_withou --"t"--> Node_without(((without)))
+          style Node_without stroke:lightgrey,color:lightgrey
+          linkStyle 37 stroke:lightgrey,color:lightgrey
+
+```
+
+(An example of slicing the trie above with the range "bit"-"thing".
+Processing only applies on boundary nodes (root, "t", "th", "thi"),
+where we throw away the transitions outside the range. Subtries like the
+ones for "n" and "o" fall completely between "b" and "t" thus are fully
+inside the range and can be processed without any further restrictions.)
+
+A trie can be used as a modifiable in-memory data structure where one
+can add and remove individual elements. It can also be constructed from
+sorted input, incrementally storing the data directly to disk and
+building an efficient read-only on-disk data structure.
+
+For more formal information about the concept and applications of tries
+and finite state automata, try [Introduction to Automata Theory,
+Languages, and Computation](https://books.google.bg/books?id=dVipBwAAQBAJ).
+There are many variations of the concept, and of the implementation of
+states and transitions that can be put to use to achieve even further
+efficiency gains; some of these will be detailed below.
+
+# Indexing with tries
+
+Since a trie is generally an ordered byte source to payload map, we can
+apply the concept directly to the components of Cassandra that are most
+affected by the inefficiency of using comparison-based structures: the
+indices.
+
+This can be done in the following way:
+
+-   When we write the index, we map each key into its byte-ordered
+    representation and create an on-disk trie of byte-ordered
+    representations of keys mapping into positions in the data file.
+
+-   When we need an exact match for a key, we create a (lazily
+    generated) byte-ordered representation of the key and look for it
+    in the trie.
+
+    -   If we find a match, we know the data file position.
+
+    -   If there is no match, there is no data associated with the key.
+
+-   When we need a greater-than/greater-or-equal match, we use the
+    byte-ordered representation to create a path that leads to the
+    first matching data position in the sstable.
+
+    -   We can then use this path to iterate the greater keys in the
+        sstable.
+
+This works, but isn't very efficient. Lookup in it is _O_(len(key)), 
+which can even mean that many seeks on disk, and we have to store
+a transition (which defines the size of the structure) for every
+non-prefix character in the dataset.
+
+We can do much better.
+
+## Trimming the fat
+
+The primary purpose of the index is to find a position in the data file
+for the given key. It needs to be able to find the correct position for
+any existing key, but there is no need for it to be exact on keys that
+are not present in the file &mdash; since our data files contain a copy of
+the key at the start of each partition, we can simply check if the key
+we are searching for matches the key at the position returned by the
+index.
+
+This allows us to use a simple optimization: instead of storing the full
+key in the index trie, we can store only a prefix of the key that is
+unique among all partitions in the table. This means that we have
+intermediate nodes in the trie only if a prefix is shared by multiple
+keys, which normally reduces the number of nodes and transitions in the
+trie to about 2*n*.
+
+```mermaid
+graph TD
+  Node_(( ))
+  style Node_ fill:darkgrey
+  Node_  --"a"--> Node_a((( )))
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+  Node_  --"t"--> Node_t(( ))
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"o"--> Node_to((( )))
+    Node_t --"r"--> Node_tr((( )))
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+```
+
+This also reduces the number of steps we need to take in the trie. In a
+well-balanced key set (such as the one where the byte-ordered key starts
+with a hash as in Murmur or Random-partitioned primary keys) the lookup
+complexity becomes _O_(log _n_) transitions[^2].
+
+[^2]: For comparison, the complexity of binary search in a sorted
+primary index is also _O_(log _n_), but in key comparisons whose
+complexity on average in a well-balanced key set is another _O_(log _n_)
+for a total _O_(log<sup>2</sup> _n_).
+
+## Taking hardware into account
+
+The point above improves the number of transitions significantly, but
+the out-of-cache efficiency is still pretty bad if we have to read a new
+disk page every time we examine a node. Fortunately we can take some
+extra care during construction to make sure we make the most of every
+disk page brought up during lookup.
+
+The idea of this is to pack wide sections of the trie in pages, so that
+every time we open a page we can be certain to be able to follow several
+transitions before leaving that page.
+
+```mermaid
+graph TD
+  subgraph p1 [ ]
+  Node_(( ))
+  style Node_ fill:darkgrey
+    Node_  --"a"--> Node_a((( )))
+    Node_  --"t"--> Node_t(( ))
+  end
+  
+  subgraph p2 [ ]
+    Node_a --"l"--> Node_al((( )))
+    Node_a --"n"--> Node_an((( )))
+      Node_an --"d"--> Node_and((( )))
+      Node_an --"y"--> Node_any((( )))
+  end
+  
+  subgraph p3 [ ]
+    Node_a --"r"--> Node_ar((( )))
+    Node_a --"s"--> Node_as((( )))
+  Node_  --"n"--> Node_n((( )))
+  end
+  
+  subgraph p4 [ ]
+  Node_  --"o"--> Node_o(( ))
+    Node_o --"f"--> Node_of((( )))
+    Node_o --"n"--> Node_on((( )))
+    Node_t --"o"--> Node_to((( )))
+  end
+  
+  subgraph p5 [ ]
+    Node_t --"h"--> Node_th(( ))
+      Node_th --"e"--> Node_the((( )))
+      Node_th --"i"--> Node_thi((( )))
+    Node_t --"r"--> Node_tr((( )))
+  end
+  
+  subgraph p6 [ ]
+    Node_t --"y"--> Node_ty((( )))
+  Node_  --"w"--> Node_w(( ))
+    Node_w --"ith"--> Node_with((( )))
+          Node_with --"o"--> Node_without((( )))
+  end
+  
+  p2 ~~~ p3 ~~~ p4 ~~~ p5 ~~~ p6
+```
+
+One way to generate something like this is to start from the root and do
+a breadth-first walk, placing the encountered nodes on disk until a page
+is filled and their target transitions in a queue for which the process
+is repeated to fill other pages.
+
+Another approach, more suitable to our application because it can be
+done as part of the incremental construction process, is to do the
+packing from the bottom up &mdash; when the incremental construction
+algorithm completes a node we do not immediately write it, but wait
+until we have formed a branch that is bigger than a page. When this
+happens we lay out the node's children (each smaller than a page but
+root of a biggest branch that would fit) and let the parent node be
+treated like a leaf from there on. In turn it will become part of a
+branch that is bigger than a page and will be laid packaged together
+with its related nodes, resulting in a picture similar to the above.
+
+In fact the bottom-up process has a little performance benefit over the
+top-down: with the top-down construction the root page is full and leaf
+pages take combinations of unrelated smaller branches; with the
+bottom-up the leaf pages take as much information as possible about a
+branch, while the root often remains unfilled. For the best possible
+out-of-cache efficiency we would prefer the set of non-leaf pages to be
+as small as possible. Having larger leaf page branches means more of the
+trie data is in the leaf branches and thus the size of that intermediate
+node set is smaller.
+
+See [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java) 
+for details on how the page-aware
+trie construction is implemented.
+
+## Storing the trie
+
+Another interesting question about the format of the trie is how one
+stores the information about the transitions in a node. If we want to
+maintain that the size of the structure is proportional to the number of
+overall transitions, we need to be able to store node transitions
+sparsely. Typically this is done using a list of transition characters
+and binary searching among them to make a transition.
+
+This binary search can theoretically be taken to use constant time
+(since the alphabet size is small and predefined), but isn't the most
+efficient operation in practice due to the unpredictable branch
+instructions necessary for its implementation. It is preferable to avoid
+it as much as possible.
+
+To do this, and to shave a few additional bytes in common cases, our
+implementation of on-disk tries uses typed nodes. A node can be:
+
+-   Final with no transitions (`PAYLOAD_ONLY`).
+
+-   Having one transition (`SINGLE`), which has to store only the
+    character and target for that transition.
+
+-   Having a binary-searched list of transitions (`SPARSE`), where the
+    number of characters, each character and the targets are stored.
+
+-   Having a consecutive range of transitions (`DENSE`), where the first
+    and last character and targets are stored, possibly including some
+    null transitions.
+
+We use one byte per node to store four bits of node type as well as four
+bits of payload information.
+
+In a well-balanced and populated trie the nodes where lookup spends most
+time (the nodes closest to the root) are `DENSE` nodes, where finding the
+target for the transition is a direct calculation from the code of the
+character. On the other hand, most of the nodes (the ones closest to the
+leaves) are `PAYLOAD_ONLY`, `SINGLE` or `SPARSE` to avoid taking any more
+space than necessary.
+
+The main objective for the trie storage format is to achieve the
+smallest possible packing (and thus smallest cache usage and fewest disk
+reads), thus we choose the type that results in the smallest
+representation of the node. `DENSE` type gets chosen naturally when its
+encoding (which avoids storing the character list but may include null
+targets) is smaller than `SPARSE`.
+
+## Pointer Sizes
+
+The next optimization we make in the storage format is based on the fact
+that most nodes in the trie are in the lower levels of the tree and thus
+close to leaves. As such, the distance between the node and its target
+transitions when laid out during the construction process is small and
+thus it is a huge win to store pointers as distances with variable size.
+
+This is even more true for the page-aware layout we use &mdash; all internal
+transitions within the page (i.e. >99% of all transitions in the trie!)
+can be stored using just an offset within the page, using just 12 bits.
+
+This is heavily used via further specialization of the node types: e.g.
+we have `DENSE_12`, `DENSE_16` to `DENSE_40` as well as `DENSE_LONG`
+subtypes which differ in the size of pointer they use.
+
+# Primary indexing in the BTI format
+
+The purpose of the primary index of an sstable is to be able to map a
+key containing partition and clustering components to a position in the
+sstable data file which holds the relevant row or the closest row with a
+greater key and enables iteration of rows from that point on.
+
+Partition keys are normally fully specified, while clustering keys are
+often given partially or via a comparison relation. They are also
+treated differently by all the infrastructure and have historically had
+different index structures; we chose to retain this distinction for the
+time being and implement similar replacement structures using tries.
+
+## Partition index implementation details
+
+The primary purpose of the partition index is to map a specified
+partition key to a row index for the partition. It also needs to support
+iteration from a (possibly partially specified) partition position. The
+description below details mapping only; iteration is a trivial
+application of the trie machinery to the described structure.
+
+In addition to wide partitions where a row index is mandatory, Cassandra
+is often used for tables where the partitions have only a
+couple of rows, including also ones where the partition key is the only
+component of the primary key, i.e. where row and partition are the same
+thing. For these situations it makes no sense to actually have a row
+index and the partition index should point directly to the data.
+
+The application of tries to Cassandra's partition index uses the trie
+infrastructure described above to create a trie mapping unique
+byte-ordered partition key prefixes to either:
+
+-   A position in the row index file which contains the index of the
+    rows within that partition, or
+
+-   A position in the data file containing the relevant partition (if a
+    row index for it is not necessary).
+
+A single table can have both indexed and non-indexed rows. For
+efficiency the partition index stores the position as a single long,
+using its sign bit to differentiate between the two options[^3]. This
+value is stored with variable length &mdash; more precisely, we use the four
+bits provided in the node type byte to store the length of the pointer.
+
+[^3]: It needs to differentiate between 0 with index and 0 without
+index, however, so we use ~pos instead of -pos to encode
+direct-to-data mappings. This still allows sign expansion
+instructions to be used to convert e.g. `int` to `long`.
+
+Lookup in this index is accomplished by converting the decorated
+partition key to its byte-ordered representation and following the
+transitions for its bytes while the trie has any. If at any point the
+trie does not offer a transition for the next byte but is not a leaf
+node, the sstable does not contain a mapping for the given key.
+
+If a leaf of the trie is reached, then the prefix of the partition key
+matches some content in the file, but we are not yet sure if it is a
+full match for the partition key. The leaf node points to a place in the
+row index or data file. In either case the first bytes at the specified
+position contain a serialization of the partition key, which we can
+compare to the key being mapped. If it matches, we have found the
+partition. If not, since the stored prefixes are unique, no data for
+this partition exists in this sstable.
+
+### Efficiency
+
+If everything is in cache this lookup is extremely efficient: it follows
+a few transitions in `DENSE` nodes plus one or two binary searches in
+`SPARSE` or `SINGLE`, and finishes with a direct comparison of a byte buffer
+with contents of a file. No object allocation or deserialization is
+necessary.
+
+If not all data is in cache, the performance of this lookup most heavily
+depends on the number of pages that must be fetched from persistent
+storage. The expectation on which this implementation is based, is that
+if an sstable is in use all non-leaf pages of the index will tend to
+remain cached. If that expectation is met, lookup will only require
+fetching one leaf index page and one data/row index page for the full
+key comparison. On a match the latter fetch will be required anyway,
+since we would want to read the data at that position.
+
+An important consideration in the design of this feature was to make
+sure there is no situation in which the trie indices perform worse than
+the earlier code, thus we should aim to do at most as many reads. The
+number of random accesses for the earlier index implementation where an
+index summary is forced in memory is one _seek_ required to start
+reading from the partition index (though usually multiple consecutive
+pages need to be read), and one seek needed to start reading the actual
+data. Since the index summary ends up being of similar size to the
+non-leaf pages of the trie index, the memory usage and number of seeks
+for the trie index on match ends up being the same but we read less data
+and do much less processing.
+
+On mismatch, though, we may be making one additional seek. However, we
+can drastically reduce the chance of mismatch, which we currently do in
+two ways:
+
+-   By using a bloom filter before lookup. The chance of getting a bloom
+    filter hit as well as a prefix match for the wrong key is pretty
+    low and gets lower with increasing sstable size.
+
+-   By storing some of the key hash bits that are not part of the token
+    at the payload node and comparing them with the mapped key's hash
+    bits.
+
+Currently we use a combination of both by default as the best performing
+option. The user can disable or choose to have a smaller bloom filter,
+and the code also supports indices that do not contain hash bits (though
+to reduce configuration complexity we do not have plans to expose that
+option).
+
+For fully cold sstables we have to perform more random fetches from disk
+than the earlier implementation, but we read less. Testing showed that
+having a bloom filter is enough to make the trie index faster; if a
+bloom filter is not present, we try going through the byte contents of
+the index file on boot to prefetch it which ends up taking not too long
+(since it is read sequentially rather than randomly) and boosting cold
+performance dramatically.
+
+### Building and early open
+
+The partition index is built using the page-aware incremental
+construction described earlier, where we also delay writing each key
+until we have seen the next so that we can find the shortest prefix that
+is enough to differentiate it from the previous and next keys (this also
+differentiates it from all others in the sstable because the contents
+are sorted). Only that prefix is written to the trie.
+
+One last complication is the support for early opening of sstables which
+allows newly-compacted tables to gradually occupy the page cache. Though
+the index building is incremental, the partially-written trie is not
+usable directly because the root of the trie as well as the path from it
+to the last written nodes is not yet present in the file.
+
+This problem can be easily overcome, though, by dumping these
+intermediate nodes to an in-memory buffer (without the need for
+page-aware packing) and forming an index by attaching this buffer at the
+end of the partially written file using 
+[`TailOverridingRebufferer`](../../../util/TailOverridingRebufferer.java).
+
+## Row index implementation details
+
+Unlike the partition index, the main use of the row index is to iterate
+from a given clustering key in forward or reverse direction (where exact
+key lookup is just a special case).
+
+Rows are often very small (they could contain a single int or no columns
+at all) and thus there is a real possibility for the row indices to
+become bigger than the data they represent. This is not a desirable
+outcome, which is part of the reason why Cassandra's row index has
+historically operated on blocks of rows rather than indexing every row
+in the partition. This is a concern we also have with the trie-based
+index, thus we also index blocks of rows (by default, a block of rows
+that is at least 16kb in size &mdash; this will be called the index
+_granularity_ below, specified by the `column_index_size_in_kb`
+`cassandra.yaml` parameter).
+
+Our row index implementation thus creates a map from clustering keys or
+prefixes to the data position at the start of the index block which is
+the earliest that could contain a key equal or greater than the given
+one. Additionally, if there is an active deletion at the beginning of
+the block, the index must specify it so that it can be taken into
+account when merging data from multiple sstables.
+
+Each index block will contain at least one key, but generally it will
+have different first and last keys. We don't store these keys, but
+instead we index the positions between blocks by storing a "separator",
+some key that is greater than the last key of the previous block and
+smaller than or equal to the first key of the next[^4]. Then, when we
+look up a given clustering, we follow its bytes as long as we can in the
+trie and we can be certain that all blocks before the closest
+less-than-or-equal entry in the trie cannot contain any data that is
+greater than or equal to the given key.
+
+[^4]: Another way to interpret this is that we index the start of each
+block only, but for efficiency we don't use the first key of the
+block as its beginning, but instead something closer to the last key
+of the previous block (but still greater than it).
+
+It may happen that the identified block actually doesn't contain any
+matching data (e.g. because the looked-up key ends up between the last
+key in the block and the separator), but this only affects efficiency as
+the iteration mechanism does not expect the data position returned by
+the index to be guaranteed to start with elements that fit the criteria;
+it would only have to walk a whole block forward to find the matching
+key.
+
+It is important to keep the number of these false positives low, and at
+the same time we aim for the smallest possible size of the index for a
+given granularity. The choice of separator affects this balance[^5]; the
+option we use, as a good tradeoff in the vein of the unique prefix
+approach used in the partition index, is to use the shortest prefix of
+the next block's beginning key that separates it from the previous
+block's end key, adjusted so that the last byte of it is 1 greater than
+that end key.
+
+[^5]: For example, the best separator for false positives is the next
+possible byte sequence after the previous block's final key, which
+is obtained by adding a 00 byte to its end. This, however, means all
+the bytes of the byte-ordered representation of this key must be
+present in the index, which inflates the index's size and lookup
+complexity.
+
+For example, if block 2 covers "something" to "somewhere" and block 3
+&mdash; "sorry" to "tease", then the sequence "son" is used as the separator
+between blocks 2 and 3. This leaves things like "sommelier" in the area
+that triggers false positives, but stores and has to walk just three
+bytes to find the starting point for iteration.
+
+### Efficiency
+
+Finding the candidate block in the trie involves walking the byte
+ordered representation of the clustering key in the trie and finding the
+closest less-than-or-equal value. The number of steps is proportional to
+the length of the separators &mdash; the lower their number the shorter that
+sequence is, though we can't expect _O_(log _n_) complexity since there may
+be many items sharing the same long prefixes (e.g. if there are long
+strings in the components of the clustering keys before the last). Even
+so, such repeating prefixes are addressed very well by the page-packing
+and `SINGLE_NOPAYLOAD_4` node type, resulting in very efficient walks.
+
+After this step we also perform a linear walk within the data file to
+find the actual start of the matching data. This is usually costlier and
+may involve object allocation and deserialization.
+
+The tradeoff between the size of the index and the time it takes to find
+the relevant rows is controlled by the index granularity. The lower it
+is, the more efficient lookup (especially exact match lookup) becomes at
+the expense of bigger index size. The 16kb default is chosen pretty
+conservatively[^6]; if users don't mind bigger indices something like 4,
+2 or 1kb granularity should be quite a bit more efficient. It is also
+possible to index every row by choosing a granularity of 0kb; at these
+settings in-cache trie-indexed sstables tend to outperform
+`ConcurrentSkipListMap` memtables for reads.
+
+[^6]: This was chosen with the aim to match the size of the trie index
+compared to the earlier version of the row index at its default
+granularity of 64kb.
+
+### Reverse lookup
+
+To perform a reverse lookup, we can use the same mechanism as above
+(with greater-than-or-equal) to find the initial block for the
+iteration. However, in the forward direction we could simply walk the
+data file to find the next rows, but this isn't possible going
+backwards.
+
+To solve this problem the index helps the iteration machinery by
+providing an iterator of index blocks in reverse order. For each index
+block the iteration walks it forward and creates a stack of all its row
+positions, then starts issuing rows by popping and examining rows from
+that stack. When the stack is exhausted it requests the previous block
+from the index and applies the same procedure there.
+
+# Code structure
+
+The implementation is mostly in two packages, `o.a.c.io.tries` contains
+the generic code to construct and read on-disk tries, and 
+`o.a.c.io.sstable.format.bti`, which implements the specifics of the
+format and the two indexes.
+
+## Building tries
+
+Tries are built from sorted keys using an [`IncrementalTrieWriter`](../../../tries/IncrementalTrieWriter.java). 
+The code contains three implementations with increasing complexity:
+- [`IncrementalTrieWriterSimple`](../../../tries/IncrementalTrieWriterSimple.java)
+  implements simple incremental construction of tries from sorted input,
+- [`IncrementalTrieWriterPageAware`](../../../tries/IncrementalTrieWriterPageAware.java)
+  adds packing of nodes to disk pages,
+- [`IncrementalDeepTrieWriterPageAware`](../../../tries/IncrementalDeepTrieWriterPageAware.java)
+  adds the ability to transition to on-heap recursion for all stages of the construction
+  process to be able to handle very large keys.
+
+Only the latter is used, but we provide (and test) the other two as a form of
+documentation.
+
+The builders take a `TrieSerializer` as parameter, which determines how the nodes
+are written. The indexes implement this using `TrieNode`, writing any payload they
+need immediately after the node serialization.
+
+## Reading tries
+
+The BTI format tries are used directly in their on-disk format. To achieve this,
+all node types are implemented as static objects in `TrieNode`. Reading nodes in
+a file is encapsulated in [`Walker`](../../../tries/Walker.java), 
+which provides a method to `go` to a specific node and use it, i.e. 
+get any associated data, search in the children list and
+follow transitions to children. It also provides functionality to find the
+mapping for a given key, floors and ceilings as well as some combinations.
+Iterating the payloads between two key bounds is implemented by 
+[`ValueIterator`](../../../tries/ValueIterator.java),
+and [`ReverseValueIterator`](../../../tries/ReverseValueIterator.java).
+
+Special care is given to prefixes to make sure the semantics of searches matches
+what the format needs.
+
+## SSTable format implementation
+
+The two indexes are implemented, respectively, by [`PartitionIndex`](PartitionIndex.java)
+and [`RowIndexReader`](RowIndexReader.java)/[`RowIndexWriter`](RowIndexWriter.java). 
+The format implementation extends the filtered
+base class and follows the structure of the BIG implementation, where
+all references to the primary index are replaced with calls to these two 
+classes.
+
+# Index file format in BTI
+
+## Trie nodes
+Implemented in [`TrieNode.java`](../../../tries/TrieNode.java)
+
+Nodes start with four bits of node type, followed by 4 payload bits
+(_pb_), which are 0 if the node has no associated payload; otherwise the
+node type gives an option to compute the starting position for the
+payload (_ppos_). The layout of the node depends on its type.
+
+`PAYLOAD_ONLY` nodes:
+
+-   4 type bits, 0
+
+-   4 payload bits
+
+-   payload if _pb_ &ne; 0, _ppos_ is _node_ + 1

Review Comment:
   Perhaps we should use "position" or "node position" rather than just "node" here to indicate it's from the starting position of the node and this is calculated to exactly "skip over" the rest of the node contents.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1184163134


##########
test/unit/org/apache/cassandra/io/tries/TrieBuilderTest.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.TailOverridingRebufferer;
+
+import static org.junit.Assert.assertEquals;
+
+public class TrieBuilderTest extends AbstractTrieTestBase
+{
+    @Test
+    public void testPartialBuild_DB1148() throws IOException

Review Comment:
   nit: Is there an OSS version of DB-1148?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181735536


##########
src/java/org/apache/cassandra/io/sstable/format/bti/TrieIndexEntry.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An entry in the row index for a partition whose rows are indexed in a trie.

Review Comment:
   nit: Worth mentioning it might also just be directly pointing to a position in the data file if there's no row index? Documentation is probably enough to manage the overloading, although maybe it would be nice to have a non-row-indexed `AbstractRowIndexEntry`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183083684


##########
src/java/org/apache/cassandra/io/sstable/format/bti/SSTableReversedIterator.java:
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import com.carrotsearch.hppc.LongStack;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.UnfilteredValidation;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.io.sstable.AbstractRowIndexEntry;
+import org.apache.cassandra.io.sstable.AbstractSSTableIterator;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+
+/**
+ * Unfiltered row iterator over a BTI SSTable that returns rows in reverse order.
+ */
+class SSTableReversedIterator extends AbstractSSTableIterator<TrieIndexEntry>
+{
+    /**
+     * The index of the slice being processed.
+     */
+    private int slice;
+
+    public SSTableReversedIterator(BtiTableReader sstable,
+                                   FileDataInput file,
+                                   DecoratedKey key,
+                                   TrieIndexEntry indexEntry,
+                                   Slices slices,
+                                   ColumnFilter columns,
+                                   FileHandle ifile)
+    {
+        super(sstable, file, key, indexEntry, slices, columns, ifile);
+    }
+
+    protected Reader createReaderInternal(TrieIndexEntry indexEntry, FileDataInput file, boolean shouldCloseFile)
+    {
+        if (indexEntry.isIndexed())
+            return new ReverseIndexedReader(indexEntry, file, shouldCloseFile);
+        else
+            return new ReverseReader(file, shouldCloseFile);
+    }
+
+    public boolean isReverseOrder()
+    {
+        return true;
+    }
+
+    protected int nextSliceIndex()
+    {
+        int next = slice;
+        slice++;
+        return slices.size() - (next + 1);
+    }
+
+    protected boolean hasMoreSlices()
+    {
+        return slice < slices.size();
+    }
+
+    /**
+     * Reverse iteration is performed by going through an index block (or the whole partition if not indexed) forwards
+     * and storing the positions of each entry that falls within the slice in a stack. Reverse iteration then pops out
+     * positions and reads the entries.
+     * <p>
+     * Note: The earlier version of this was constructing an in-memory view of the block instead, which gives better
+     * performance on bigger queries and index blocks (due to not having to read disk again). With the lower
+     * granularity of the tries it makes better sense to store as little as possible as the beginning of the block
+     * should very rarely be in other page/chunk cache locations. This has the benefit of being able to answer small
+     * queries (esp. LIMIT 1) faster and with less GC churn.
+     */
+    private class ReverseReader extends AbstractReader
+    {
+        final LongStack rowOffsets = new LongStack();
+        RangeTombstoneMarker blockOpenMarker, blockCloseMarker;
+        private Unfiltered next = null;
+        private boolean foundLessThan;
+        private long startPos = -1;
+
+        private ReverseReader(FileDataInput file, boolean shouldCloseFile)
+        {
+            super(file, shouldCloseFile);
+        }
+
+        public void setForSlice(Slice slice) throws IOException
+        {
+            // read full row and filter
+            if (startPos == -1)
+                startPos = file.getFilePointer();
+            else
+                seekToPosition(startPos);
+
+            fillOffsets(slice, true, true, Long.MAX_VALUE);
+        }
+
+        protected boolean hasNextInternal() throws IOException
+        {
+            if (next != null)
+                return true;
+            next = computeNext();
+            return next != null;
+        }
+
+        protected Unfiltered nextInternal() throws IOException
+        {
+            if (!hasNextInternal())
+                throw new NoSuchElementException();
+
+            Unfiltered toReturn = next;
+            next = null;
+            return toReturn;
+        }
+
+        private Unfiltered computeNext() throws IOException
+        {
+            Unfiltered toReturn;
+            do
+            {
+                if (blockCloseMarker != null)
+                {
+                    toReturn = blockCloseMarker;
+                    blockCloseMarker = null;
+                    return toReturn;
+                }
+                while (!rowOffsets.isEmpty())
+                {
+                    seekToPosition(rowOffsets.pop());
+                    boolean hasNext = deserializer.hasNext();
+                    assert hasNext;
+                    toReturn = deserializer.readNext();
+                    UnfilteredValidation.maybeValidateUnfiltered(toReturn, metadata(), key, sstable);
+                    // We may get empty row for the same reason expressed on UnfilteredSerializer.deserializeOne.
+                    if (!toReturn.isEmpty())
+                        return toReturn;
+                }
+            }
+            while (!foundLessThan && advanceIndexBlock());
+
+            // open marker to be output only as slice is finished
+            if (blockOpenMarker != null)
+            {
+                toReturn = blockOpenMarker;
+                blockOpenMarker = null;
+                return toReturn;
+            }
+            return null;
+        }
+
+        protected boolean advanceIndexBlock() throws IOException
+        {
+            return false;
+        }
+
+        void fillOffsets(Slice slice, boolean filterStart, boolean filterEnd, long stopPosition) throws IOException
+        {
+            filterStart &= !slice.start().equals(ClusteringBound.BOTTOM);
+            filterEnd &= !slice.end().equals(ClusteringBound.TOP);
+            long currentPosition = -1;
+
+            ClusteringBound start = slice.start();
+            currentPosition = file.getFilePointer();

Review Comment:
   ```suggestion
               int currentPosition = file.getFilePointer();
   ```
   Doesn't seem to be a point in initializing to `-1` above?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183081154


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIterator.java:
##########
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Throwables;
+
+import static org.apache.cassandra.utils.FBUtilities.immutableListWithFilteredNulls;
+
+class PartitionIterator extends PartitionIndex.IndexPosIterator implements KeyReader
+{
+    private final PartitionIndex partitionIndex;
+    private final IPartitioner partitioner;
+    private final PartitionPosition limit;
+    private final int exclusiveLimit;
+    private final FileHandle dataFile;
+    private final FileHandle rowIndexFile;
+
+    private FileDataInput dataInput;
+    private FileDataInput indexInput;
+
+    private DecoratedKey currentKey;
+    private TrieIndexEntry currentEntry;
+    private DecoratedKey nextKey;
+    private TrieIndexEntry nextEntry;
+
+    @SuppressWarnings({ "resource", "RedundantSuppression" })
+    static PartitionIterator create(PartitionIndex partitionIndex, IPartitioner partitioner, FileHandle rowIndexFile, FileHandle dataFile,
+                                    PartitionPosition left, int inclusiveLeft, PartitionPosition right, int exclusiveRight) throws IOException
+    {
+        PartitionIterator partitionIterator = null;
+        PartitionIndex partitionIndexCopy = null;
+        FileHandle dataFileCopy = null;
+        FileHandle rowIndexFileCopy = null;
+
+        try
+        {
+            partitionIndexCopy = partitionIndex.sharedCopy();
+            dataFileCopy = dataFile.sharedCopy();
+            rowIndexFileCopy = rowIndexFile.sharedCopy();
+
+            partitionIterator = new PartitionIterator(partitionIndexCopy, partitioner, rowIndexFileCopy, dataFileCopy, left, right, exclusiveRight);
+
+            partitionIterator.readNext();
+            // Because the index stores prefixes, the first value can be in any relationship with the left bound.
+            if (partitionIterator.nextKey != null && !(partitionIterator.nextKey.compareTo(left) > inclusiveLeft))
+            {
+                partitionIterator.readNext();
+            }
+            partitionIterator.advance();

Review Comment:
   So to make sure I understand, we `readNext()` to load the next key and index entry, then `advance()` actually assigns them to the "current" key and entry, which will be accessible via `key()` and `entry()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181876510


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);

Review Comment:
   nit: Untested code path?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185538661


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException
+    {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        // Check that long repeated strings don't cause stack overflow
+        // Test both normal completion and partial construction.
+        Thread t = new Thread(null, () ->
+        {
+            try
+            {
+                File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+                SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build());
+                List<DecoratedKey> list = Lists.newArrayList();
+                String longString = "";
+                for (int i = 0; i < PageAware.PAGE_SIZE + 99; ++i)
+                {
+                    longString += i;
+                }
+                IPartitioner partitioner = ByteOrderedPartitioner.instance;
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "A")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "B")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "C")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "D")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "E")));
+
+                FileHandle.Builder fhBuilder = new FileHandle.Builder(file)
+                                               .bufferSize(PageAware.PAGE_SIZE)
+                                               .withChunkCache(ChunkCache.instance);
+                try (PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder))
+                {
+                    int i = 0;
+                    for (i = 0; i < 3; ++i)
+                        builder.addEntry(list.get(i), i);
+
+                    writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                    AtomicInteger callCount = new AtomicInteger();
+
+                    final int addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             index.close();
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+
+                    for (; i < list.size(); ++i)
+                        builder.addEntry(list.get(i), i);
+                    builder.complete();
+
+                    try (PartitionIndex index = PartitionIndex.load(fhBuilder, partitioner, true))
+                    {
+                        checkIteration(list, list.size(), index);
+                    }
+                }
+                future.complete(null);
+            }
+            catch (Throwable err)
+            {
+                future.completeExceptionally(err);
+            }
+        }, "testThread", 32 * 1024);
+
+        t.start();
+        future.join();
+    }
+
+    class JumpingFile extends SequentialWriter
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingFile(File file, SequentialWriterOption option, long... cutoffsAndOffsets)
+        {
+            super(file, option);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public long position()
+        {
+            return jumped(super.position(), cutoffs, offsets);
+        }
+    }
+
+    class JumpingRebufferer extends WrappingRebufferer
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingRebufferer(Rebufferer source, long... cutoffsAndOffsets)
+        {
+            super(source);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            long pos;
+
+            int idx = Arrays.binarySearch(offsets, position);
+            if (idx < 0)
+                idx = -2 - idx;
+            pos = position;
+            if (idx >= 0)
+                pos = pos - offsets[idx] + cutoffs[idx];
+
+            super.rebuffer(pos);
+
+            if (idx < cutoffs.length - 1 && buffer.limit() + offset > cutoffs[idx + 1])
+                buffer.limit((int) (cutoffs[idx + 1] - offset));
+            if (idx >= 0)
+                offset = offset - cutoffs[idx] + offsets[idx];
+
+            return this;
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return jumped(wrapped.fileLength(), cutoffs, offsets);
+        }
+
+        @Override
+        public String toString()
+        {
+            return Arrays.toString(cutoffs) + Arrays.toString(offsets);
+        }
+    }
+
+    public class PartitionIndexJumping extends PartitionIndex
+    {
+        final long[] cutoffsAndOffsets;
+
+        public PartitionIndexJumping(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last,
+                                     long... cutoffsAndOffsets)
+        {
+            super(fh, trieRoot, keyCount, first, last);
+            this.cutoffsAndOffsets = cutoffsAndOffsets;
+        }
+
+        @Override
+        protected Rebufferer instantiateRebufferer()
+        {
+            return new JumpingRebufferer(super.instantiateRebufferer(), cutoffsAndOffsets);
+        }
+    }
+
+    long jumped(long pos, long[] cutoffs, long[] offsets)
+    {
+        int idx = Arrays.binarySearch(cutoffs, pos);
+        if (idx < 0)
+            idx = -2 - idx;
+        if (idx < 0)
+            return pos;
+        return pos - cutoffs[idx] + offsets[idx];
+    }
+
+    @Test
+    public void testPointerGrowth() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            long[] cutoffsAndOffsets = new long[]{
+            2 * 4096, 1L << 16,
+            4 * 4096, 1L << 24,
+            6 * 4096, 1L << 31,
+            8 * 4096, 1L << 32,
+            10 * 4096, 1L << 33,
+            12 * 4096, 1L << 34,
+            14 * 4096, 1L << 40,
+            16 * 4096, 1L << 42
+            };
+
+            List<DecoratedKey> list = Lists.newArrayList();
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeJumpingWriter(file, cutoffsAndOffsets);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+
+                for (int i = 0; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                long root = builder.complete();
+
+                try (FileHandle fh = fhBuilder.complete();
+                     PartitionIndex index = new PartitionIndexJumping(fh, root, COUNT, null, null, cutoffsAndOffsets);
+                     Analyzer analyzer = new Analyzer(index))
+                {
+                    checkIteration(list, list.size(), index);
+
+                    analyzer.run();
+                    if (analyzer.countPerType.elementSet().size() < 7)
+                    {
+                        Assert.fail("Expecting at least 7 different node types, got " + analyzer.countPerType.elementSet().size() + "\n" + analyzer.countPerType);
+                    }
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDumpTrieToFile() throws IOException
+    {
+        File file = FileUtils.createTempFile("testDumpTrieToFile", "index");
+
+        ArrayList<DecoratedKey> list = Lists.newArrayList();
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.DEFAULT);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+            for (int i = 0; i < 1000; i++)
+            {
+                DecoratedKey key = generateRandomKey();
+                list.add(key);
+            }
+            Collections.sort(list);
+
+            for (int i = 0; i < 1000; ++i)
+                builder.addEntry(list.get(i), i);
+            long root = builder.complete();
+
+            try (FileHandle fh = fhBuilder.complete();
+                 PartitionIndex index = new PartitionIndex(fh, root, 1000, null, null))
+            {
+                File dump = FileUtils.createTempFile("testDumpTrieToFile", "dumpedTrie");
+                index.dumpTrie(dump.toString());
+                String dumpContent = String.join("\n", Files.readAllLines(dump.toPath()));
+                logger.info("Dumped trie: \n{}", dumpContent);
+                assertFalse(dumpContent.isEmpty());
+            }
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static class Analyzer extends PartitionIndex.Reader
+    {
+        Multiset<TrieNode> countPerType = HashMultiset.create();
+
+        public Analyzer(PartitionIndex index)
+        {
+            super(index);
+        }
+
+        public void run()
+        {
+            run(root);
+        }
+
+        void run(long node)
+        {
+            go(node);
+
+            countPerType.add(nodeType);
+
+            int tr = transitionRange();
+            for (int i = 0; i < tr; ++i)
+            {
+                long child = transition(i);
+                if (child == NONE)
+                    continue;
+                run(child);
+                go(node);
+            }
+        }
+    }
+
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateLongKeysIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateLongKey);
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateSequentialIndex(int size) throws IOException
+    {
+        return generateIndex(size, new Supplier<DecoratedKey>()
+        {
+            long i = 0;
+
+            public DecoratedKey get()
+            {
+                return sequentialKey(i++);
+            }
+        });
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndexWithZeroCopy(int size, int firstKeyOffset, int lastKeyOffset) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey, firstKeyOffset, lastKeyOffset, true);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator) throws IOException
+    {
+        return generateIndex(size, keyGenerator, 0, size - 1, false);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator, int firstKeyOffset, int lastKeyOffset, boolean hasZeroCopy) throws IOException

Review Comment:
   Feels like that would be a problem for `testZeroCopyOffsets()`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185539060


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)

Review Comment:
   nit: `keys` unused



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blambov commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "blambov (via GitHub)" <gi...@apache.org>.
blambov commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185840805


##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java:
##########
@@ -221,73 +219,4 @@ public void testOldReadsNew(String oldV, String newV) throws IOException
         }
     }
 
-    @Test
-    public void pendingRepairCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me").forEach(v -> assertFalse(format.getVersion(v).hasPendingRepair()));
-            Arrays.asList("na", "nb", "nc").forEach(v -> assertTrue(format.getVersion(v).hasPendingRepair()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void originatingHostCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "na").forEach(v -> assertFalse(format.getVersion(v).hasOriginatingHostId()));
-            Arrays.asList("me", "nb").forEach(v -> assertTrue(format.getVersion(v).hasOriginatingHostId()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void improvedMinMaxCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasImprovedMinMax()));
-            Arrays.asList("nc", "oa").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasImprovedMinMax()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void legacyMinMaxCompatiblity()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("oa").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasLegacyMinMax()));
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb", "nc").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasLegacyMinMax()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }
-
-    @Test
-    public void partitionLevelDeletionPresenceMarkerCompatibility()
-    {
-        if (format == BigFormat.instance)
-        {
-            Arrays.asList("ma", "mb", "mc", "md", "me", "na", "nb").forEach(v -> assertFalse(BigFormat.instance.getVersion(v).hasPartitionLevelDeletionsPresenceMarker()));
-            Arrays.asList("nc", "oa").forEach(v -> assertTrue(BigFormat.instance.getVersion(v).hasPartitionLevelDeletionsPresenceMarker()));
-        }
-        else
-        {
-            throw Util.testMustBeImplementedForSSTableFormat();
-        }
-    }

Review Comment:
   Yes, to separate instances for BIG and BTI formats.



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ * <ul>
+ *   <li> calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   <li> delaying writing any part of a completed node until its branch size is above the page size
+ *   <li> laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   <li> adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ * </ul>
+ * <p>
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ * <p>
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * <pre>
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * </pre>
+ * where lines denote page boundaries.
+ * <p>
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * <p>
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * <p>
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -163,6 +168,14 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
 
     public static SSTableReader sstable(int generation, int size, boolean keepRef, long firstToken, long lastToken, int level, ColumnFamilyStore cfs, int minLocalDeletionTime, long timestamp)
     {
+        try (DataOutputStreamPlus out = tempFile.newOutputStream(File.WriteMode.OVERWRITE))
+        {
+            out.write(new byte[10]);

Review Comment:
   Makes sure we have a non-empty file to refer to. Moving this to the static initializer to see if we really need to do it every time.



##########
test/unit/org/apache/cassandra/io/tries/AbstractTrieTestBase.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.function.BiFunction;
+
+import org.junit.Before;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+@RunWith(Parameterized.class)
+abstract public class AbstractTrieTestBase
+{
+    @Parameterized.Parameter(0)
+    public TestClass writerClass;
+
+    enum TestClass
+    {
+        SIMPLE(IncrementalTrieWriterSimple::new),
+        PAGE_AWARE(IncrementalTrieWriterPageAware::new),
+        PAGE_AWARE_DEEP_ON_STACK((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 256)),
+        PAGE_AWARE_DEEP_ON_HEAP((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 0)),
+        PAGE_AWARE_DEEP_MIXED((serializer, dest) -> new IncrementalDeepTrieWriterPageAware<>(serializer, dest, 2));
+
+        final BiFunction<TrieSerializer<Integer, DataOutputPlus>, DataOutputPlus, IncrementalTrieWriter<Integer>> constructor;
+        TestClass(BiFunction<TrieSerializer<Integer, DataOutputPlus>, DataOutputPlus, IncrementalTrieWriter<Integer>> constructor)
+        {
+            this.constructor = constructor;
+        }
+    }
+
+    @Parameterized.Parameters(name = "{index}: trie writer class={0}")
+    public static Collection<Object[]> data()
+    {
+        return Arrays.asList(new Object[]{ TestClass.SIMPLE },
+                             new Object[]{ TestClass.PAGE_AWARE },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_ON_STACK },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_ON_HEAP },
+                             new Object[]{ TestClass.PAGE_AWARE_DEEP_MIXED });
+    }
+
+    protected final static Logger logger = LoggerFactory.getLogger(TrieBuilderTest.class);
+    protected final static int BASE = 80;
+
+    protected boolean dump = false;
+    protected int payloadSize = 0;
+
+    @Before
+    public void beforeTest()
+    {
+        dump = false;
+        payloadSize = 0;
+    }
+
+    IncrementalTrieWriter<Integer> newTrieWriter(TrieSerializer<Integer, DataOutputPlus> serializer, DataOutputPlus out)
+    {
+        return writerClass.constructor.apply(serializer, out);
+    }
+
+    protected final TrieSerializer<Integer, DataOutputPlus> serializer = new TrieSerializer<Integer, DataOutputPlus>()
+    {
+        public int sizeofNode(SerializationNode<Integer> node, long nodePosition)
+        {
+            return TrieNode.typeFor(node, nodePosition).sizeofNode(node) + payloadSize;
+        }
+
+        public void write(DataOutputPlus dataOutput, SerializationNode<Integer> node, long nodePosition) throws IOException
+        {
+            if (dump)
+                logger.info("Writing at {} type {} size {}: {}", Long.toHexString(nodePosition), TrieNode.typeFor(node, nodePosition), TrieNode.typeFor(node, nodePosition).sizeofNode(node), node);
+            TrieNode.typeFor(node, nodePosition).serialize(dataOutput, node, node.payload() != null ? node.payload() : 0, nodePosition);

Review Comment:
   Yes, added comment.



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/schema/MockSchema.java:
##########
@@ -219,6 +233,55 @@ public static SSTableReader sstable(int generation, int size, boolean keepRef, l
                 return reader;
             }
         }
+        else if (format == BtiFormat.instance)
+        {
+            Set<Component> components = ImmutableSet.of(Components.DATA, BtiFormat.Components.PARTITION_INDEX, BtiFormat.Components.ROW_INDEX, Components.FILTER, Components.TOC);
+            for (Component component : components)
+            {
+                File file = descriptor.fileFor(component);
+                file.createFileIfNotExists();
+            }
+            // .complete() with size to make sstable.onDiskLength work
+            try (FileHandle fileHandle = new FileHandle.Builder(tempFile).bufferSize(size).withLengthOverride(size).complete())
+            {
+                if (size > 0)
+                {
+                    try
+                    {
+                        File file = descriptor.fileFor(Components.DATA);
+                        Util.setFileLength(file, size);
+                    }
+                    catch (IOException e)
+                    {
+                        throw new RuntimeException(e);
+                    }
+                }

Review Comment:
   Extracted



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException
+        {
+            nodePosition = nodePosition + node.branchSize;
+            assert dest.position() == nodePosition
+                    : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+            serializer.write(dest, node, nodePosition);
+
+            assert dest.position() == nodePosition + node.nodeSize
+                   || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                    : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+
+            node.filePos = nodePosition;
+        }
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalDeepTrieWriterPageAware.java:
##########
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * This class is a variant of {@link IncrementalTrieWriterPageAware} which is able to build even very deep
+ * tries. While the parent class uses recursion for clarity, it may end up with stack overflow for tries with
+ * very long keys. This implementation can switch processing from stack to heap at a certain depth (provided
+ * as a constructor param).
+ * <p>
+ * This class intentionally repeats code present in the parent class, both in the in-stack and on-heap versions
+ * of each of the three implemented recursive operations. Removing this repetition can cause higher stack usage
+ * and thus stack overflow failures.
+ */
+@NotThreadSafe
+public class IncrementalDeepTrieWriterPageAware<VALUE> extends IncrementalTrieWriterPageAware<VALUE>
+{
+    private final int maxRecursionDepth;
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest, int maxRecursionDepth)
+    {
+        super(trieSerializer, dest);
+        this.maxRecursionDepth = maxRecursionDepth;
+    }
+
+    public IncrementalDeepTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        this(trieSerializer, dest, 64);
+    }
+
+    /**
+     * Simple framework for executing recursion using on-heap linked trace to avoid stack overruns.
+     */
+    static abstract class Recursion<NODE>
+    {
+        final Recursion<NODE> parent;
+        final NODE node;
+        final Iterator<NODE> childIterator;
+
+        Recursion(NODE node, Iterator<NODE> childIterator, Recursion<NODE> parent)
+        {
+            this.parent = parent;
+            this.node = node;
+            this.childIterator = childIterator;
+        }
+
+        /**
+         * Make a child Recursion object for the given node and initialize it as necessary to continue processing
+         * with it.
+         * <p>
+         * May return null if the recursion does not need to continue inside the child branch.
+         */
+        abstract Recursion<NODE> makeChild(NODE child);
+
+        /**
+         * Complete the processing this Recursion object.
+         * <p>
+         * Note: this method is not called for the nodes for which makeChild() returns null.
+         */
+        abstract void complete() throws IOException;
+
+        /**
+         * Complete processing of the given child (possibly retrieve data to apply to any accumulation performed
+         * in this Recursion object).
+         * <p>
+         * This is called when processing a child completes, including when recursion inside the child branch
+         * is skipped by makeChild() returning null.
+         */
+        void completeChild(NODE child)
+        {}
+
+        /**
+         * Recursive process, in depth-first order, the branch rooted at this recursion node.
+         * <p>
+         * Returns this.
+         */
+        Recursion<NODE> process() throws IOException
+        {
+            Recursion<NODE> curr = this;
+
+            while (true)
+            {
+                if (curr.childIterator.hasNext())
+                {
+                    NODE child = curr.childIterator.next();
+                    Recursion<NODE> childRec = curr.makeChild(child);
+                    if (childRec != null)
+                        curr = childRec;
+                    else
+                        curr.completeChild(child);
+                }
+                else
+                {
+                    curr.complete();
+                    Recursion<NODE> currParent = curr.parent;
+                    if (currParent == null)
+                        return curr;
+                    currParent.completeChild(curr.node);
+                    curr = currParent;
+                }
+            }
+        }
+    }
+
+    @Override
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        return recalcTotalSizeRecursiveOnStack(node, nodePosition, 0);
+    }
+
+    private int recalcTotalSizeRecursiveOnStack(Node<VALUE> node, long nodePosition, int depth) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+            {
+                if (depth < maxRecursionDepth)
+                    sz += recalcTotalSizeRecursiveOnStack(child, nodePosition + sz, depth + 1);
+                else
+                    sz += recalcTotalSizeRecursiveOnHeap(child, nodePosition + sz);
+            }
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    private int recalcTotalSizeRecursiveOnHeap(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+            new RecalcTotalSizeRecursion(node, null, nodePosition).process();
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    class RecalcTotalSizeRecursion extends Recursion<Node<VALUE>>
+    {
+        final long nodePosition;
+        int sz;
+
+        RecalcTotalSizeRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent, long nodePosition)
+        {
+            super(node, node.children.iterator(), parent);
+            sz = 0;
+            this.nodePosition = nodePosition;
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.hasOutOfPageInBranch)
+                return new RecalcTotalSizeRecursion(child, this, nodePosition + sz);
+            else
+                return null;
+        }
+
+        @Override
+        void complete()
+        {
+            node.branchSize = sz;
+        }
+
+        @Override
+        void completeChild(Node<VALUE> child)
+        {
+            // This will be called for nodes that were recursively processed as well as the ones that weren't.
+
+            // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+            // we need to recalculate the size if either flag is set.
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                long childPosition = this.nodePosition + sz;
+                child.nodeSize = serializer.sizeofNode(child, childPosition + child.branchSize);
+            }
+
+            sz += child.branchSize + child.nodeSize;
+        }
+    }
+
+    @Override
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        return writeRecursiveOnStack(node, 0);
+    }
+
+    private long writeRecursiveOnStack(Node<VALUE> node, int depth) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+            {
+                if (depth < maxRecursionDepth)
+                    child.filePos = writeRecursiveOnStack(child, depth + 1);
+                else
+                    child.filePos = writeRecursiveOnHeap(child);
+            }
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+        : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+               || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+        : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    private long writeRecursiveOnHeap(Node<VALUE> node) throws IOException
+    {
+        return new WriteRecursion(node, null).process().node.filePos;
+    }
+
+    class WriteRecursion extends Recursion<Node<VALUE>>
+    {
+        long nodePosition;
+
+        WriteRecursion(Node<VALUE> node, Recursion<Node<VALUE>> parent)
+        {
+            super(node, node.children.iterator(), parent);
+            nodePosition = dest.position();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+                return new WriteRecursion(child, this);
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException
+        {
+            nodePosition = nodePosition + node.branchSize;
+            assert dest.position() == nodePosition
+                    : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+            serializer.write(dest, node, nodePosition);
+
+            assert dest.position() == nodePosition + node.nodeSize
+                   || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                    : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+
+            node.filePos = nodePosition;
+        }
+    }
+
+    @Override
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        return writePartialRecursiveOnStack(node, dest, baseOffset, 0);
+    }
+
+    private long writePartialRecursiveOnStack(Node<VALUE> node, DataOutputPlus dest, long baseOffset, int depth) throws IOException
+    {
+        long startPosition = dest.position() + baseOffset;
+
+        List<Node<VALUE>> childrenToClear = new ArrayList<>();
+        for (Node<VALUE> child : node.children)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                if (depth < maxRecursionDepth)
+                    child.filePos = writePartialRecursiveOnStack(child, dest, baseOffset, depth + 1);
+                else
+                    child.filePos = writePartialRecursiveOnHeap(child, dest, baseOffset);
+            }
+        }
+
+        long nodePosition = dest.position() + baseOffset;
+
+        if (node.hasOutOfPageInBranch)
+        {
+            // Update the branch size with the size of what we have just written. This may be used by the node's
+            // maxPositionDelta, and it's a better approximation for later fitting calculations.
+            node.branchSize = (int) (nodePosition - startPosition);
+        }
+
+        serializer.write(dest, node, nodePosition);
+
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+        {
+            // Update the node size with what we have just seen. It's a better approximation for later fitting
+            // calculations.
+            long endPosition = dest.position() + baseOffset;
+            node.nodeSize = (int) (endPosition - nodePosition);
+        }
+
+        for (Node<VALUE> child : childrenToClear)
+            child.filePos = -1;
+        return nodePosition;
+    }
+
+    private long writePartialRecursiveOnHeap(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException
+    {
+        new WritePartialRecursion(node, dest, baseOffset).process();
+        long pos = node.filePos;
+        node.filePos = -1;
+        return pos;
+    }
+
+    class WritePartialRecursion extends Recursion<Node<VALUE>>
+    {
+        final DataOutputPlus dest;
+        final long baseOffset;
+        final long startPosition;
+        final List<Node<VALUE>> childrenToClear;
+
+        WritePartialRecursion(Node<VALUE> node, WritePartialRecursion parent)
+        {
+            super(node, node.children.iterator(), parent);
+            this.dest = parent.dest;
+            this.baseOffset = parent.baseOffset;
+            this.startPosition = dest.position() + baseOffset;
+            childrenToClear = new ArrayList<>();
+        }
+
+        WritePartialRecursion(Node<VALUE> node, DataOutputPlus dest, long baseOffset)
+        {
+            super(node, node.children.iterator(), null);
+            this.dest = dest;
+            this.baseOffset = baseOffset;
+            this.startPosition = dest.position() + baseOffset;
+            childrenToClear = new ArrayList<>();
+        }
+
+        @Override
+        Recursion<Node<VALUE>> makeChild(Node<VALUE> child)
+        {
+            if (child.filePos == -1)
+            {
+                childrenToClear.add(child);
+                return new WritePartialRecursion(child, this);
+            }
+            else
+                return null;
+        }
+
+        @Override
+        void complete() throws IOException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());

Review Comment:
   Done



##########
src/java/org/apache/cassandra/io/tries/IncrementalTrieWriterPageAware.java:
##########
@@ -0,0 +1,464 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.TreeSet;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * Incremental builders of on-disk tries which packs trie stages into disk cache pages.
+ *
+ * The incremental core is as in {@link IncrementalTrieWriterSimple}, which this augments by:
+ * <ul>
+ *   <li> calculating branch sizes reflecting the amount of data that needs to be written to store the trie
+ *     branch rooted at each node
+ *   <li> delaying writing any part of a completed node until its branch size is above the page size
+ *   <li> laying out (some of) its children branches (each smaller than a page) to be contained within a page
+ *   <li> adjusting the branch size to reflect the fact that the children are now written (i.e. removing their size)
+ * </ul>
+ * <p>
+ * The process is bottom-up, i.e. pages are packed at the bottom and the root page is usually smaller.
+ * This may appear less efficient than a top-down process which puts more information in the top pages that
+ * tend to stay in cache, but in both cases performing a search will usually require an additional disk read
+ * for the leaf page. When we maximize the amount of relevant data that read brings by using the bottom-up
+ * process, we have practically the same efficiency with smaller intermediate page footprint, i.e. fewer data
+ * to keep in cache.
+ * <p>
+ * As an example, taking a sample page size fitting 4 nodes, a simple trie would be split like this:
+ * <pre>
+ * Node 0 |
+ *   -a-> | Node 1
+ *        |   -s-> Node 2
+ *        |          -k-> Node 3 (payload 1)
+ *        |          -s-> Node 4 (payload 2)
+ *        -----------------------------------
+ *   -b-> Node 5 |
+ *          -a-> |Node 6
+ *               |  -n-> Node 7
+ *               |         -k-> Node 8 (payload 3)
+ *               |                -s-> Node 9 (payload 4)
+ * </pre>
+ * where lines denote page boundaries.
+ * <p>
+ * The process itself will start by adding "ask" which adds three nodes after the root to the stack. Adding "ass"
+ * completes Node 3, setting its branch a size of 1 and replaces it on the stack with Node 4.
+ * The step of adding "bank" starts by completing Node 4 (size 1), Node 2 (size 3), Node 1 (size 4), then adds 4 more
+ * nodes to the stack. Adding "banks" descends one more node.
+ * <p>
+ * The trie completion step completes nodes 9 (size 1), 8 (size 2), 7 (size 3), 6 (size 4), 5 (size 5). Since the size
+ * of node 5 is above the page size, the algorithm lays out its children. Nodes 6, 7, 8, 9 are written in order. The
+ * size of node 5 is now just the size of it individually, 1. The process continues with completing Node 0 (size 6).
+ * This is bigger than the page size, so some of its children need to be written. The algorithm takes the largest,
+ * Node 1, and lays it out with its children in the file. Node 0 now has an adjusted size of 2 which is below the
+ * page size, and we can continue the process.
+ * <p>
+ * Since this was the root of the trie, the current page is padded and the remaining nodes 0, 5 are written.
+ */
+@NotThreadSafe
+public class IncrementalTrieWriterPageAware<VALUE>
+extends IncrementalTrieWriterBase<VALUE, DataOutputPlus, IncrementalTrieWriterPageAware.Node<VALUE>>
+implements IncrementalTrieWriter<VALUE>
+{
+    final int maxBytesPerPage;
+
+    private final static Comparator<Node<?>> BRANCH_SIZE_COMPARATOR = (l, r) ->
+    {
+        // Smaller branches first.
+        int c = Integer.compare(l.branchSize + l.nodeSize, r.branchSize + r.nodeSize);
+        if (c != 0)
+            return c;
+
+        // Then order by character, which serves several purposes:
+        // - enforces inequality to make sure equal sizes aren't treated as duplicates,
+        // - makes sure the item we use for comparison key comes greater than all equal-sized nodes,
+        // - orders equal sized items so that most recently processed (and potentially having closer children) comes
+        //   last and is thus the first one picked for layout.
+        c = Integer.compare(l.transition, r.transition);
+
+        assert c != 0 || l == r;
+        return c;
+    };
+
+    IncrementalTrieWriterPageAware(TrieSerializer<VALUE, ? super DataOutputPlus> trieSerializer, DataOutputPlus dest)
+    {
+        super(trieSerializer, dest, new Node<>((byte) 0));
+        this.maxBytesPerPage = dest.maxBytesInPage();
+    }
+
+    @Override
+    public void reset()
+    {
+        reset(new Node<>((byte) 0));
+    }
+
+    @Override
+    Node<VALUE> performCompletion() throws IOException
+    {
+        Node<VALUE> root = super.performCompletion();
+
+        int actualSize = recalcTotalSize(root, dest.position());
+        int bytesLeft = dest.bytesLeftInPage();
+        if (actualSize > bytesLeft)
+        {
+            if (actualSize <= maxBytesPerPage)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                // position changed, recalculate again
+                actualSize = recalcTotalSize(root, dest.position());
+            }
+
+            if (actualSize > bytesLeft)
+            {
+                // Still greater. Lay out children separately.
+                layoutChildren(root);
+
+                // Pad if needed and place.
+                if (root.nodeSize > dest.bytesLeftInPage())
+                {
+                    dest.padToPageBoundary();
+                    // Recalculate again as pointer size may have changed, triggering assertion in writeRecursive.
+                    recalcTotalSize(root, dest.position());
+                }
+            }
+        }
+
+
+        root.finalizeWithPosition(write(root));
+        return root;
+    }
+
+    @Override
+    void complete(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        int branchSize = 0;
+        for (Node<VALUE> child : node.children)
+            branchSize += child.branchSize + child.nodeSize;
+
+        node.branchSize = branchSize;
+
+        int nodeSize = serializer.sizeofNode(node, dest.position());
+        if (nodeSize + branchSize < maxBytesPerPage)
+        {
+            // Good. This node and all children will (most probably) fit page.
+            node.nodeSize = nodeSize;
+            node.hasOutOfPageChildren = false;
+            node.hasOutOfPageInBranch = false;
+
+            for (Node<VALUE> child : node.children)
+                if (child.filePos != -1)
+                    node.hasOutOfPageChildren = true;
+                else if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+                    node.hasOutOfPageInBranch = true;
+
+            return;
+        }
+
+        // Cannot fit. Lay out children; The current node will be marked as one with out-of-page children.
+        layoutChildren(node);
+    }
+
+    private void layoutChildren(Node<VALUE> node) throws IOException
+    {
+        assert node.filePos == -1;
+
+        NavigableSet<Node<VALUE>> children = node.getChildrenWithUnsetPosition();
+
+        int bytesLeft = dest.bytesLeftInPage();
+        Node<VALUE> cmp = new Node<>(256); // goes after all equal-sized unplaced nodes (whose transition character is 0-255)
+        cmp.nodeSize = 0;
+        while (!children.isEmpty())
+        {
+            cmp.branchSize = bytesLeft;
+            Node<VALUE> child = children.headSet(cmp, true).pollLast();    // grab biggest that could fit
+            if (child == null)
+            {
+                dest.padToPageBoundary();
+                bytesLeft = maxBytesPerPage;
+                child = children.pollLast();       // just biggest
+            }
+
+            assert child != null;
+            if (child.hasOutOfPageChildren || child.hasOutOfPageInBranch)
+            {
+                // We didn't know what size this branch will actually need to be, node's children may be far.
+                // We now know where we would place it, so let's reevaluate size.
+                int actualSize = recalcTotalSize(child, dest.position());
+                if (actualSize > bytesLeft)
+                {
+                    if (bytesLeft == maxBytesPerPage)
+                    {
+                        // Branch doesn't even fit in a page.
+
+                        // Note: In this situation we aren't actually making the best choice as the layout should have
+                        // taken place at the child (which could have made the current parent small enough to fit).
+                        // This is not trivial to fix but should be very rare.
+
+                        layoutChildren(child);
+                        bytesLeft = dest.bytesLeftInPage();
+
+                        assert (child.filePos == -1);
+                    }
+
+                    // Doesn't fit, but that's probably because we don't have a full page. Put it back with the new
+                    // size and retry when we do have enough space.
+                    children.add(child);
+                    continue;
+                }
+            }
+
+            child.finalizeWithPosition(write(child));
+            bytesLeft = dest.bytesLeftInPage();
+        }
+
+        // The sizing below will use the branch size, so make sure it's set.
+        node.branchSize = 0;
+        node.hasOutOfPageChildren = true;
+        node.hasOutOfPageInBranch = false;
+        node.nodeSize = serializer.sizeofNode(node, dest.position());
+    }
+
+    protected int recalcTotalSize(Node<VALUE> node, long nodePosition) throws IOException
+    {
+        if (node.hasOutOfPageInBranch)
+        {
+            int sz = 0;
+            for (Node<VALUE> child : node.children)
+                sz += recalcTotalSize(child, nodePosition + sz);
+            node.branchSize = sz;
+        }
+
+        // The sizing below will use the branch size calculated above. Since that can change on out-of-page in branch,
+        // we need to recalculate the size if either flag is set.
+        if (node.hasOutOfPageChildren || node.hasOutOfPageInBranch)
+            node.nodeSize = serializer.sizeofNode(node, nodePosition + node.branchSize);
+
+        return node.branchSize + node.nodeSize;
+    }
+
+    protected long write(Node<VALUE> node) throws IOException
+    {
+        long nodePosition = dest.position();
+        for (Node<VALUE> child : node.children)
+            if (child.filePos == -1)
+                child.filePos = write(child);
+
+        nodePosition += node.branchSize;
+        assert dest.position() == nodePosition
+                : "Expected node position to be " + nodePosition + " but got " + dest.position() + " after writing children.\n" + dumpNode(node, dest.position());
+
+        serializer.write(dest, node, nodePosition);
+
+        assert dest.position() == nodePosition + node.nodeSize
+                || dest.paddedPosition() == dest.position() // For PartitionIndexTest.testPointerGrowth where position may jump on page boundaries.
+                : "Expected node position to be " + (nodePosition + node.nodeSize) + " but got " + dest.position() + " after writing node, nodeSize " + node.nodeSize + ".\n" + dumpNode(node, nodePosition);
+        return nodePosition;
+    }
+
+    protected String dumpNode(Node<VALUE> node, long nodePosition)
+    {
+        StringBuilder res = new StringBuilder(String.format("At %,d(%x) type %s child count %s nodeSize %,d branchSize %,d %s%s%n",
+                                                            nodePosition, nodePosition,
+                                                            TrieNode.typeFor(node, nodePosition), node.childCount(), node.nodeSize, node.branchSize,
+                                                            node.hasOutOfPageChildren ? "C" : "",
+                                                            node.hasOutOfPageInBranch ? "B" : ""));
+        for (Node<VALUE> child : node.children)
+            res.append(String.format("Child %2x at %,d(%x) type %s child count %s size %s nodeSize %,d branchSize %,d %s%s%n",
+                                     child.transition & 0xFF,
+                                     child.filePos,
+                                     child.filePos,
+                                     child.children != null ? TrieNode.typeFor(child, child.filePos) : "n/a",
+                                     child.children != null ? child.childCount() : "n/a",
+                                     child.children != null ? serializer.sizeofNode(child, child.filePos) : "n/a",
+                                     child.nodeSize,
+                                     child.branchSize,
+                                     child.hasOutOfPageChildren ? "C" : "",
+                                     child.hasOutOfPageInBranch ? "B" : ""));
+
+        return res.toString();
+    }
+
+    @Override
+    public PartialTail makePartialRoot() throws IOException
+    {
+        // The expectation is that the partial tail will be in memory, so we don't bother with page-fitting.
+        // We could also send some completed children to disk, but that could make suboptimal layout choices, so we'd
+        // rather not. Just write anything not written yet to a buffer, from bottom to top, and we're done.
+        try (DataOutputBuffer buf = new DataOutputBuffer())
+        {
+            PTail tail = new PTail();
+            // Readers ask rebufferers for page-aligned positions, so make sure tail starts at one.
+            // "Padding" of the cutoff point may leave some unaddressable space in the constructed file view.
+            // Nothing will point to it, though, so that's fine.
+            tail.cutoff = dest.paddedPosition();
+            tail.count = count;
+            tail.root = writePartial(stack.getFirst(), buf, tail.cutoff);
+            tail.tail = buf.asNewBuffer();
+            return tail;
+        }
+    }
+
+    protected long writePartial(Node<VALUE> node, DataOutputPlus dest, long baseOffset) throws IOException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/tries/WalkerTest.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.tries;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.TailOverridingRebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
+
+@SuppressWarnings("unchecked")
+public class WalkerTest extends AbstractTrieTestBase
+{
+    @Test
+    public void testWithoutBounds() throws IOException
+    {
+        DataOutputBuffer buf = new DataOutputBufferPaged();
+        IncrementalTrieWriter<Integer> builder = makeTrie(buf);
+        long rootPos = builder.complete();
+
+        Rebufferer source = new ByteBufRebufferer(buf.asNewBuffer());
+
+        InternalIterator it = new InternalIterator(source, rootPos);
+
+        DataOutputBuffer dumpBuf = new DataOutputBuffer();
+        it.dumpTrie(new PrintStream(dumpBuf), (buf1, payloadPos, payloadFlags) -> String.format("%d/%d", payloadPos, payloadFlags));
+        logger.info("Trie dump: \n{}", new String(dumpBuf.getData()));
+        logger.info("Trie toString: {}", it);
+
+        it.goMax(rootPos);
+        assertEquals(7, it.payloadFlags());
+        assertEquals(TrieNode.Types.PAYLOAD_ONLY.ordinal, it.nodeTypeOrdinal());
+        assertEquals(1, it.nodeSize());
+        assertFalse(it.hasChildren());
+
+        it.goMin(rootPos);
+        assertEquals(1, it.payloadFlags());
+        assertEquals(TrieNode.Types.PAYLOAD_ONLY.ordinal, it.nodeTypeOrdinal());
+        assertEquals(1, it.nodeSize());
+        assertFalse(it.hasChildren());
+
+        assertEquals(-1, it.follow(source("151")));
+        assertEquals(2, it.payloadFlags());
+
+        assertEquals('3', it.follow(source("135")));
+
+        assertEquals('3', it.followWithGreater(source("135")));
+        it.goMin(it.greaterBranch);
+        assertEquals(2, it.payloadFlags());
+
+        assertEquals('3', it.followWithLesser(source("135")));
+        it.goMax(it.lesserBranch);
+        assertEquals(1, it.payloadFlags());
+
+        assertEquals(3, (Object) it.prefix(source("155"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertNull(it.prefix(source("516"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(5, (Object) it.prefix(source("5151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(1, (Object) it.prefix(source("1151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+
+        assertEquals(3, (Object) it.prefixAndNeighbours(source("155"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertNull(it.prefixAndNeighbours(source("516"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(5, (Object) it.prefixAndNeighbours(source("5151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        assertEquals(1, (Object) it.prefixAndNeighbours(source("1151"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+
+        assertEquals(3, (Object) it.prefixAndNeighbours(source("1555"), (walker, payloadPosition, payloadFlags) -> payloadFlags));
+        it.goMax(it.lesserBranch);
+        assertEquals(2, it.payloadFlags());
+        it.goMin(it.greaterBranch);
+        assertEquals(4, it.payloadFlags());
+    }
+
+    @Test
+    public void testWithBounds() throws IOException
+    {
+        DataOutputBuffer buf = new DataOutputBufferPaged();
+        IncrementalTrieWriter<Integer> builder = makeTrie(buf);
+        long rootPos = builder.complete();
+
+        Rebufferer source = new ByteBufRebufferer(buf.asNewBuffer());
+
+        InternalIterator it = new InternalIterator(source, rootPos, source("151"), source("515"), false);
+        long pos;
+        assertNotEquals(-1, pos = it.nextPayloadedNode());
+        assertEquals(3, TrieNode.at(buf.asNewBuffer(), (int) pos).payloadFlags(buf.asNewBuffer(), (int) pos));

Review Comment:
   Not at all, but there is more to it. We must also return "151" for a "15151" query with `admitPrefix`.
   
   I added some clarifying JavaDoc and more tests, and corrected some errors when `admitPrefix == false`, which we never actually used.



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException

Review Comment:
   Removed catch clause.



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);

Review Comment:
   Done



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()

Review Comment:
   Added



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException
+    {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        // Check that long repeated strings don't cause stack overflow
+        // Test both normal completion and partial construction.
+        Thread t = new Thread(null, () ->
+        {
+            try
+            {
+                File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+                SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build());
+                List<DecoratedKey> list = Lists.newArrayList();
+                String longString = "";
+                for (int i = 0; i < PageAware.PAGE_SIZE + 99; ++i)
+                {
+                    longString += i;
+                }
+                IPartitioner partitioner = ByteOrderedPartitioner.instance;
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "A")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "B")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "C")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "D")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "E")));
+
+                FileHandle.Builder fhBuilder = new FileHandle.Builder(file)
+                                               .bufferSize(PageAware.PAGE_SIZE)
+                                               .withChunkCache(ChunkCache.instance);
+                try (PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder))
+                {
+                    int i = 0;
+                    for (i = 0; i < 3; ++i)
+                        builder.addEntry(list.get(i), i);
+
+                    writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                    AtomicInteger callCount = new AtomicInteger();
+
+                    final int addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             index.close();
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+
+                    for (; i < list.size(); ++i)
+                        builder.addEntry(list.get(i), i);
+                    builder.complete();
+
+                    try (PartitionIndex index = PartitionIndex.load(fhBuilder, partitioner, true))
+                    {
+                        checkIteration(list, list.size(), index);
+                    }
+                }
+                future.complete(null);
+            }
+            catch (Throwable err)
+            {
+                future.completeExceptionally(err);
+            }
+        }, "testThread", 32 * 1024);
+
+        t.start();
+        future.join();
+    }
+
+    class JumpingFile extends SequentialWriter
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingFile(File file, SequentialWriterOption option, long... cutoffsAndOffsets)
+        {
+            super(file, option);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public long position()
+        {
+            return jumped(super.position(), cutoffs, offsets);
+        }
+    }
+
+    class JumpingRebufferer extends WrappingRebufferer
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingRebufferer(Rebufferer source, long... cutoffsAndOffsets)
+        {
+            super(source);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            long pos;
+
+            int idx = Arrays.binarySearch(offsets, position);
+            if (idx < 0)
+                idx = -2 - idx;
+            pos = position;
+            if (idx >= 0)
+                pos = pos - offsets[idx] + cutoffs[idx];
+
+            super.rebuffer(pos);
+
+            if (idx < cutoffs.length - 1 && buffer.limit() + offset > cutoffs[idx + 1])
+                buffer.limit((int) (cutoffs[idx + 1] - offset));
+            if (idx >= 0)
+                offset = offset - cutoffs[idx] + offsets[idx];
+
+            return this;
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return jumped(wrapped.fileLength(), cutoffs, offsets);
+        }
+
+        @Override
+        public String toString()
+        {
+            return Arrays.toString(cutoffs) + Arrays.toString(offsets);
+        }
+    }
+
+    public class PartitionIndexJumping extends PartitionIndex
+    {
+        final long[] cutoffsAndOffsets;
+
+        public PartitionIndexJumping(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last,
+                                     long... cutoffsAndOffsets)
+        {
+            super(fh, trieRoot, keyCount, first, last);
+            this.cutoffsAndOffsets = cutoffsAndOffsets;
+        }
+
+        @Override
+        protected Rebufferer instantiateRebufferer()
+        {
+            return new JumpingRebufferer(super.instantiateRebufferer(), cutoffsAndOffsets);
+        }
+    }
+
+    long jumped(long pos, long[] cutoffs, long[] offsets)
+    {
+        int idx = Arrays.binarySearch(cutoffs, pos);
+        if (idx < 0)
+            idx = -2 - idx;
+        if (idx < 0)
+            return pos;
+        return pos - cutoffs[idx] + offsets[idx];
+    }
+
+    @Test
+    public void testPointerGrowth() throws IOException

Review Comment:
   Removed catch clause.



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException
+    {
+        CompletableFuture<Void> future = new CompletableFuture<>();
+        // Check that long repeated strings don't cause stack overflow
+        // Test both normal completion and partial construction.
+        Thread t = new Thread(null, () ->
+        {
+            try
+            {
+                File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+                SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build());
+                List<DecoratedKey> list = Lists.newArrayList();
+                String longString = "";
+                for (int i = 0; i < PageAware.PAGE_SIZE + 99; ++i)
+                {
+                    longString += i;
+                }
+                IPartitioner partitioner = ByteOrderedPartitioner.instance;
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "A")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "B")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "C")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "D")));
+                list.add(partitioner.decorateKey(ByteBufferUtil.bytes(longString + "E")));
+
+                FileHandle.Builder fhBuilder = new FileHandle.Builder(file)
+                                               .bufferSize(PageAware.PAGE_SIZE)
+                                               .withChunkCache(ChunkCache.instance);
+                try (PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder))
+                {
+                    int i = 0;
+                    for (i = 0; i < 3; ++i)
+                        builder.addEntry(list.get(i), i);
+
+                    writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                    AtomicInteger callCount = new AtomicInteger();
+
+                    final int addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             index.close();
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+
+                    for (; i < list.size(); ++i)
+                        builder.addEntry(list.get(i), i);
+                    builder.complete();
+
+                    try (PartitionIndex index = PartitionIndex.load(fhBuilder, partitioner, true))
+                    {
+                        checkIteration(list, list.size(), index);
+                    }
+                }
+                future.complete(null);
+            }
+            catch (Throwable err)
+            {
+                future.completeExceptionally(err);
+            }
+        }, "testThread", 32 * 1024);
+
+        t.start();
+        future.join();
+    }
+
+    class JumpingFile extends SequentialWriter
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingFile(File file, SequentialWriterOption option, long... cutoffsAndOffsets)
+        {
+            super(file, option);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public long position()
+        {
+            return jumped(super.position(), cutoffs, offsets);
+        }
+    }
+
+    class JumpingRebufferer extends WrappingRebufferer
+    {
+        long[] cutoffs;
+        long[] offsets;
+
+        JumpingRebufferer(Rebufferer source, long... cutoffsAndOffsets)
+        {
+            super(source);
+            assert (cutoffsAndOffsets.length & 1) == 0;
+            cutoffs = new long[cutoffsAndOffsets.length / 2];
+            offsets = new long[cutoffs.length];
+            for (int i = 0; i < cutoffs.length; ++i)
+            {
+                cutoffs[i] = cutoffsAndOffsets[i * 2];
+                offsets[i] = cutoffsAndOffsets[i * 2 + 1];
+            }
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            long pos;
+
+            int idx = Arrays.binarySearch(offsets, position);
+            if (idx < 0)
+                idx = -2 - idx;
+            pos = position;
+            if (idx >= 0)
+                pos = pos - offsets[idx] + cutoffs[idx];
+
+            super.rebuffer(pos);
+
+            if (idx < cutoffs.length - 1 && buffer.limit() + offset > cutoffs[idx + 1])
+                buffer.limit((int) (cutoffs[idx + 1] - offset));
+            if (idx >= 0)
+                offset = offset - cutoffs[idx] + offsets[idx];
+
+            return this;
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return jumped(wrapped.fileLength(), cutoffs, offsets);
+        }
+
+        @Override
+        public String toString()
+        {
+            return Arrays.toString(cutoffs) + Arrays.toString(offsets);
+        }
+    }
+
+    public class PartitionIndexJumping extends PartitionIndex
+    {
+        final long[] cutoffsAndOffsets;
+
+        public PartitionIndexJumping(FileHandle fh, long trieRoot, long keyCount, DecoratedKey first, DecoratedKey last,
+                                     long... cutoffsAndOffsets)
+        {
+            super(fh, trieRoot, keyCount, first, last);
+            this.cutoffsAndOffsets = cutoffsAndOffsets;
+        }
+
+        @Override
+        protected Rebufferer instantiateRebufferer()
+        {
+            return new JumpingRebufferer(super.instantiateRebufferer(), cutoffsAndOffsets);
+        }
+    }
+
+    long jumped(long pos, long[] cutoffs, long[] offsets)
+    {
+        int idx = Arrays.binarySearch(cutoffs, pos);
+        if (idx < 0)
+            idx = -2 - idx;
+        if (idx < 0)
+            return pos;
+        return pos - cutoffs[idx] + offsets[idx];
+    }
+
+    @Test
+    public void testPointerGrowth() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            long[] cutoffsAndOffsets = new long[]{
+            2 * 4096, 1L << 16,
+            4 * 4096, 1L << 24,
+            6 * 4096, 1L << 31,
+            8 * 4096, 1L << 32,
+            10 * 4096, 1L << 33,
+            12 * 4096, 1L << 34,
+            14 * 4096, 1L << 40,
+            16 * 4096, 1L << 42
+            };
+
+            List<DecoratedKey> list = Lists.newArrayList();
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeJumpingWriter(file, cutoffsAndOffsets);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+
+                for (int i = 0; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                long root = builder.complete();
+
+                try (FileHandle fh = fhBuilder.complete();
+                     PartitionIndex index = new PartitionIndexJumping(fh, root, COUNT, null, null, cutoffsAndOffsets);
+                     Analyzer analyzer = new Analyzer(index))
+                {
+                    checkIteration(list, list.size(), index);
+
+                    analyzer.run();
+                    if (analyzer.countPerType.elementSet().size() < 7)
+                    {
+                        Assert.fail("Expecting at least 7 different node types, got " + analyzer.countPerType.elementSet().size() + "\n" + analyzer.countPerType);
+                    }
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDumpTrieToFile() throws IOException
+    {
+        File file = FileUtils.createTempFile("testDumpTrieToFile", "index");
+
+        ArrayList<DecoratedKey> list = Lists.newArrayList();
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = new SequentialWriter(file, SequentialWriterOption.DEFAULT);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+            for (int i = 0; i < 1000; i++)
+            {
+                DecoratedKey key = generateRandomKey();
+                list.add(key);
+            }
+            Collections.sort(list);
+
+            for (int i = 0; i < 1000; ++i)
+                builder.addEntry(list.get(i), i);
+            long root = builder.complete();
+
+            try (FileHandle fh = fhBuilder.complete();
+                 PartitionIndex index = new PartitionIndex(fh, root, 1000, null, null))
+            {
+                File dump = FileUtils.createTempFile("testDumpTrieToFile", "dumpedTrie");
+                index.dumpTrie(dump.toString());
+                String dumpContent = String.join("\n", Files.readAllLines(dump.toPath()));
+                logger.info("Dumped trie: \n{}", dumpContent);
+                assertFalse(dumpContent.isEmpty());
+            }
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static class Analyzer extends PartitionIndex.Reader
+    {
+        Multiset<TrieNode> countPerType = HashMultiset.create();
+
+        public Analyzer(PartitionIndex index)
+        {
+            super(index);
+        }
+
+        public void run()
+        {
+            run(root);
+        }
+
+        void run(long node)
+        {
+            go(node);
+
+            countPerType.add(nodeType);
+
+            int tr = transitionRange();
+            for (int i = 0; i < tr; ++i)
+            {
+                long child = transition(i);
+                if (child == NONE)
+                    continue;
+                run(child);
+                go(node);
+            }
+        }
+    }
+
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateLongKeysIndex(int size) throws IOException
+    {
+        return generateIndex(size, this::generateLongKey);
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateSequentialIndex(int size) throws IOException
+    {
+        return generateIndex(size, new Supplier<DecoratedKey>()
+        {
+            long i = 0;
+
+            public DecoratedKey get()
+            {
+                return sequentialKey(i++);
+            }
+        });
+    }
+
+    private Pair<List<DecoratedKey>, PartitionIndex> generateRandomIndexWithZeroCopy(int size, int firstKeyOffset, int lastKeyOffset) throws IOException
+    {
+        return generateIndex(size, this::generateRandomKey, firstKeyOffset, lastKeyOffset, true);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator) throws IOException
+    {
+        return generateIndex(size, keyGenerator, 0, size - 1, false);
+    }
+
+    Pair<List<DecoratedKey>, PartitionIndex> generateIndex(int size, Supplier<DecoratedKey> keyGenerator, int firstKeyOffset, int lastKeyOffset, boolean hasZeroCopy) throws IOException

Review Comment:
   The test is not useful without partial zero-copy support, removed.



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)

Review Comment:
   Removed



##########
test/unit/org/apache/cassandra/io/sstable/format/bti/RowIndexTest.java:
##########
@@ -0,0 +1,517 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.io.sstable.format.bti.RowIndexReader.IndexInfo;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.DataOutputStreamPlus;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class RowIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(RowIndexTest.class);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static final Random RANDOM;
+
+    static
+    {
+        long seed = System.currentTimeMillis();
+        logger.info("seed = " + seed);
+        RANDOM = new Random(seed);
+
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    static final ClusteringComparator comparator = new ClusteringComparator(UUIDType.instance);
+    static final long END_MARKER = 1L << 40;
+    static final int COUNT = 8192;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    @Test
+    public void testSingletons() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        RowIndexReader summary = random.right;
+        List<ClusteringPrefix<?>> keys = random.left;
+        for (int i = 0; i < COUNT; i++)
+        {
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(i))).offset);
+        }
+        summary.close();
+    }
+
+    @Test
+    public void testSpans() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexQuads(COUNT);
+        RowIndexReader summary = random.right;
+        List<ClusteringPrefix<?>> keys = random.left;
+        int missCount = 0;
+        IndexInfo ii;
+        for (int i = 0; i < COUNT; i++)
+        {
+            // These need to all be within the span
+            assertEquals(i, (ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 1)))).offset);
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 2))).offset);
+            assertEquals(i, summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i + 3))).offset);
+
+            // check other data
+            assertEquals(i + 2, ii.openDeletion.markedForDeleteAt());
+            assertEquals(i - 3, ii.openDeletion.localDeletionTime());
+
+            // before entry. hopefully here, but could end up in prev if matches prevMax too well
+            ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * i)));
+            if (ii.offset != i)
+            {
+                ++missCount;
+                assertEquals(i - 1, ii.offset);
+            }
+        }
+        ii = summary.separatorFloor(comparator.asByteComparable(keys.get(4 * COUNT)));
+        if (ii.offset != END_MARKER)
+        {
+            ++missCount;
+            assertEquals(COUNT - 1, ii.offset);
+        }
+        ii = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.BOTTOM));
+        assertEquals(0, ii.offset);
+
+        ii = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.TOP));
+        assertEquals(END_MARKER, ii.offset);
+
+        summary.close();
+        if (missCount > COUNT / 5)
+            logger.error("Unexpectedly high miss count: {}/{}", missCount, COUNT);
+    }
+
+    File file;
+    DataOutputStreamPlus dos;
+    RowIndexWriter writer;
+    FileHandle fh;
+    long root;
+
+    @After
+    public void cleanUp()
+    {
+        FileUtils.closeQuietly(dos);
+        FileUtils.closeQuietly(writer);
+        FileUtils.closeQuietly(fh);
+    }
+
+    public RowIndexTest() throws IOException
+    {
+        this(FileUtils.createTempFile("ColumnTrieReaderTest", ""));
+    }
+
+    RowIndexTest(File file) throws IOException
+    {
+        this(file, new SequentialWriter(file, SequentialWriterOption.newBuilder().finishOnClose(true).build()));
+    }
+
+    RowIndexTest(File file, DataOutputStreamPlus dos) throws IOException
+    {
+        this.file = file;
+        this.dos = dos;
+
+        // write some junk
+        dos.writeUTF("JUNK");
+        dos.writeUTF("JUNK");
+
+        writer = new RowIndexWriter(comparator, dos);
+    }
+
+    public void complete() throws IOException
+    {
+        root = writer.complete(END_MARKER);
+        dos.writeUTF("JUNK");
+        dos.writeUTF("JUNK");
+        dos.close();
+        dos = null;
+    }
+
+    public RowIndexReader completeAndRead() throws IOException
+    {
+        complete();
+
+        FileHandle.Builder builder = new FileHandle.Builder(file).mmapped(accessMode == Config.DiskAccessMode.mmap);
+        fh = builder.complete();
+        try (RandomAccessReader rdr = fh.createReader())
+        {
+            assertEquals("JUNK", rdr.readUTF());
+            assertEquals("JUNK", rdr.readUTF());
+        }
+        return new RowIndexReader(fh, root);
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        ClusteringPrefix<?> key = Clustering.EMPTY;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try (RowIndexReader summary = completeAndRead())
+        {
+            IndexInfo i = summary.min();
+            assertEquals(42, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.BOTTOM));
+            assertEquals(42, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(ClusteringBound.TOP));
+            assertEquals(END_MARKER, i.offset);
+
+            i = summary.separatorFloor(comparator.asByteComparable(key));
+            assertEquals(42, i.offset);
+        }
+    }
+
+    @Test
+    public void testAddDuplicateEmptyThrow() throws Exception
+    {
+        ClusteringPrefix<?> key = Clustering.EMPTY;
+        Throwable t = null;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key, key, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testAddDuplicateThrow() throws Exception
+    {
+        ClusteringPrefix<?> key = generateRandomKey();
+        Throwable t = null;
+        writer.add(key, key, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key, key, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testAddOutOfOrderThrow() throws Exception
+    {
+        ClusteringPrefix<?> key1 = generateRandomKey();
+        ClusteringPrefix<?> key2 = generateRandomKey();
+        while (comparator.compare(key1, key2) <= 0) // make key2 smaller than 1
+            key2 = generateRandomKey();
+
+        Throwable t = null;
+        writer.add(key1, key1, new IndexInfo(42, DeletionTime.LIVE));
+        try
+        {
+            writer.add(key2, key2, new IndexInfo(43, DeletionTime.LIVE));
+            try (RowIndexReader summary = completeAndRead())
+            {
+                // failing path
+            }
+        }
+        catch (AssertionError e)
+        {
+            // correct path
+            t = e;
+            logger.info("Got " + e.getMessage());
+        }
+        Assert.assertNotNull("Should throw an assertion error.", t);
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        // This is not too relevant: due to the way we construct separators we can't be good enough on the left side.
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        List<ClusteringPrefix<?>> keys = random.left;
+
+        for (int i = 0; i < 500; ++i)
+        {
+            boolean exactLeft = RANDOM.nextBoolean();
+            boolean exactRight = RANDOM.nextBoolean();
+            ClusteringPrefix<?> left = exactLeft ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+            ClusteringPrefix<?> right = exactRight ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+            if (comparator.compare(right, left) < 0)
+            {
+                ClusteringPrefix<?> t = left;
+                left = right;
+                right = t;
+                boolean b = exactLeft;
+                exactLeft = exactRight;
+                exactRight = b;
+            }
+
+            try (RowIndexReverseIterator iter = new RowIndexReverseIterator(fh, root, comparator.asByteComparable(left), comparator.asByteComparable(right)))
+            {
+                IndexInfo indexInfo = iter.nextIndexInfo();
+                if (indexInfo == null)
+                {
+                    int idx = Collections.binarySearch(keys, right, comparator);
+                    if (idx < 0)
+                        idx = -2 - idx; // less than or equal
+                    if (idx <= 0)
+                        continue;
+                    assertTrue(comparator.asByteComparable(left) + " <= "
+                               + comparator.asByteComparable(keys.get(idx)) + " <= "
+                               + comparator.asByteComparable(right) + " but " + idx + " wasn't iterated.",
+                               comparator.compare(left, keys.get(idx - 1)) > 0);
+                    continue;
+                }
+
+                int idx = (int) indexInfo.offset;
+                if (indexInfo.offset == END_MARKER)
+                    idx = keys.size();
+                if (idx > 0)
+                    assertTrue(comparator.compare(right, keys.get(idx - 1)) > 0);
+                if (idx < keys.size() - 1)
+                    assertTrue(comparator.compare(right, keys.get(idx + 1)) < 0);
+                if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                    assertEquals(right, keys.get(idx));
+                while (true)
+                {
+                    --idx;
+                    IndexInfo ii = iter.nextIndexInfo();
+                    if (ii == null)
+                        break;
+                    assertEquals(idx, (int) ii.offset);
+                }
+                ++idx; // seek at last returned
+                if (idx < keys.size() - 1)
+                    assertTrue(comparator.compare(left, keys.get(idx + 1)) < 0);
+                // Because of the way we build the index (using non-prefix separator) we are usually going to miss the last item.
+                if (idx >= 2)
+                    assertTrue(comparator.compare(left, keys.get(idx - 2)) > 0);
+            }
+            catch (AssertionError e)
+            {
+                logger.error(e.getMessage(), e);
+                ClusteringPrefix<?> ll = left;
+                ClusteringPrefix<?> rr = right;
+                logger.info(keys.stream()
+                                .filter(x -> comparator.compare(ll, x) <= 0 && comparator.compare(x, rr) <= 0)
+                                .map(clustering -> comparator.asByteComparable(clustering))
+                                .map(bc -> bc.byteComparableAsString(VERSION))
+                                .collect(Collectors.joining(", ")));
+                logger.info("Left {}{} Right {}{}", comparator.asByteComparable(left), exactLeft ? "#" : "", comparator.asByteComparable(right), exactRight ? "#" : "");
+                try (RowIndexReverseIterator iter2 = new RowIndexReverseIterator(fh, root, comparator.asByteComparable(left), comparator.asByteComparable(right)))
+                {
+                    IndexInfo ii;
+                    while ((ii = iter2.nextIndexInfo()) != null)
+                    {
+                        logger.info(comparator.asByteComparable(keys.get((int) ii.offset)).toString());
+                    }
+                    logger.info("Left {}{} Right {}{}", comparator.asByteComparable(left), exactLeft ? "#" : "", comparator.asByteComparable(right), exactRight ? "#" : "");
+                }
+                throw e;
+            }
+        }
+    }
+
+    @Test
+    public void testReverseIteration() throws IOException
+    {
+        Pair<List<ClusteringPrefix<?>>, RowIndexReader> random = generateRandomIndexSingletons(COUNT);
+        List<ClusteringPrefix<?>> keys = random.left;
+
+        for (int i = 0; i < 1000; ++i)
+        {
+            boolean exactRight = RANDOM.nextBoolean();
+            ClusteringPrefix<?> right = exactRight ? keys.get(RANDOM.nextInt(keys.size())) : generateRandomKey();
+
+            int idx = 0;
+            try (RowIndexReverseIterator iter = new RowIndexReverseIterator(fh, root, ByteComparable.EMPTY, comparator.asByteComparable(right)))
+            {
+                IndexInfo indexInfo = iter.nextIndexInfo();
+                if (indexInfo == null)
+                {
+                    idx = Collections.binarySearch(keys, right, comparator);
+                    if (idx < 0)
+                        idx = -2 - idx; // less than or equal
+                    assertTrue(comparator.asByteComparable(keys.get(idx)) + " <= "
+                               + comparator.asByteComparable(right) + " but " + idx + " wasn't iterated.",
+                               idx < 0);

Review Comment:
   Yes, it can, if key is lower than the first one in the collection.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] dcapwell commented on pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#issuecomment-1504044743

   removed requested change in favor of [CASSANDRA-18441](https://issues.apache.org/jira/browse/CASSANDRA-18441).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167179972


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReaderLoadingBuilder.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.format.CompressionInfoComponent;
+import org.apache.cassandra.io.sstable.format.FilterComponent;
+import org.apache.cassandra.io.sstable.format.SSTableReaderLoadingBuilder;
+import org.apache.cassandra.io.sstable.format.StatsComponent;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.metrics.TableMetrics;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.FilterFactory;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.Throwables;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+public class BtiTableReaderLoadingBuilder extends SSTableReaderLoadingBuilder<BtiTableReader, BtiTableReader.Builder>
+{
+    private final static Logger logger = LoggerFactory.getLogger(BtiTableReaderLoadingBuilder.class);
+
+    private FileHandle.Builder dataFileBuilder;
+    private FileHandle.Builder partitionIndexFileBuilder;
+    private FileHandle.Builder rowIndexFileBuilder;
+
+    public BtiTableReaderLoadingBuilder(SSTable.Builder<?, ?> builder)
+    {
+        super(builder);
+    }
+
+    @Override
+    public KeyReader buildKeyReader(TableMetrics tableMetrics) throws IOException
+    {
+        StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.HEADER, MetadataType.VALIDATION);
+        return createKeyReader(statsComponent.statsMetadata());
+    }
+
+    private KeyReader createKeyReader(StatsMetadata statsMetadata) throws IOException
+    {
+        checkNotNull(statsMetadata);
+
+        try (PartitionIndex index = PartitionIndex.load(partitionIndexFileBuilder(), tableMetadataRef.getLocal().partitioner, false);
+             CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components);
+             FileHandle dFile = dataFileBuilder(statsMetadata).withCompressionMetadata(compressionMetadata).complete();
+             FileHandle riFile = rowIndexFileBuilder().complete())
+        {
+            return PartitionIterator.create(index,
+                                            tableMetadataRef.getLocal().partitioner,
+                                            riFile,
+                                            dFile);
+        }
+    }
+
+    @Override
+    protected void openComponents(BtiTableReader.Builder builder, SSTable.Owner owner, boolean validate, boolean online) throws IOException
+    {
+        try
+        {
+            StatsComponent statsComponent = StatsComponent.load(descriptor, MetadataType.STATS, MetadataType.VALIDATION, MetadataType.HEADER);
+            builder.setSerializationHeader(statsComponent.serializationHeader(builder.getTableMetadataRef().getLocal()));
+            assert !online || builder.getSerializationHeader() != null;
+
+            builder.setStatsMetadata(statsComponent.statsMetadata());
+            ValidationMetadata validationMetadata = statsComponent.validationMetadata();
+            validatePartitioner(builder.getTableMetadataRef().getLocal(), validationMetadata);
+
+            boolean filterNeeded = online;
+            if (filterNeeded)
+                builder.setFilter(loadFilter(validationMetadata));
+            boolean rebuildFilter = filterNeeded && builder.getFilter() == null;
+
+            if (descriptor.version.hasKeyRange())
+            {
+                IPartitioner partitioner = tableMetadataRef.getLocal().partitioner;
+                builder.setFirst(partitioner.decorateKey(builder.getStatsMetadata().firstKey));
+                builder.setLast(partitioner.decorateKey(builder.getStatsMetadata().lastKey));
+            }
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX) && builder.getComponents().contains(Components.ROW_INDEX) && rebuildFilter)
+            {
+                @SuppressWarnings("resource")
+                IFilter filter = buildBloomFilter(statsComponent.statsMetadata());
+
+                if (filter != null)
+                {
+                    builder.setFilter(filter);
+
+                    if (online)
+                        FilterComponent.save(filter, descriptor, false);
+                }
+            }
+
+            if (builder.getFilter() == null)
+                builder.setFilter(FilterFactory.AlwaysPresent);
+
+            if (builder.getComponents().contains(Components.ROW_INDEX))
+                builder.setRowIndexFile(rowIndexFileBuilder().complete());
+
+            if (builder.getComponents().contains(Components.PARTITION_INDEX))
+            {
+                builder.setPartitionIndex(openPartitionIndex(builder.getFilter().isInformative()));
+                builder.setFirst(builder.getPartitionIndex().firstKey());
+                builder.setLast(builder.getPartitionIndex().lastKey());
+            }
+
+            try (CompressionMetadata compressionMetadata = CompressionInfoComponent.maybeLoad(descriptor, components))
+            {
+                builder.setDataFile(dataFileBuilder(builder.getStatsMetadata()).withCompressionMetadata(compressionMetadata).complete());
+            }
+        }
+        catch (IOException | RuntimeException | Error ex)
+        {
+            // in case of failure, close only those components which have been opened in this try-catch block
+            Throwables.closeAndAddSuppressed(ex, builder.getPartitionIndex(), builder.getRowIndexFile(), builder.getDataFile(), builder.getFilter());
+            throw ex;
+        }
+    }
+
+    private IFilter buildBloomFilter(StatsMetadata statsMetadata) throws IOException
+    {
+        IFilter bf = null;
+
+        try (KeyReader keyReader = createKeyReader(statsMetadata))
+        {
+            if (keyReader == null)

Review Comment:
   nit: `keyReader` can't be null here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167171132


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableVerifier.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.LocalPartitioner;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.KeyReader;
+import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SortedTableVerifier;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+
+public class BtiTableVerifier extends SortedTableVerifier<BtiTableReader> implements IVerifier
+{
+    public BtiTableVerifier(ColumnFamilyStore cfs, BtiTableReader sstable, OutputHandler outputHandler, boolean isOffline, Options options)
+    {
+        super(cfs, sstable, outputHandler, isOffline, options);
+    }
+
+    public void verify()
+    {
+        verifySSTableVersion();
+
+        verifySSTableMetadata();
+
+        verifyIndex();
+
+        verifyBloomFilter();
+
+        if (options.checkOwnsTokens && !isOffline && !(cfs.getPartitioner() instanceof LocalPartitioner))
+        {
+            if (verifyOwnedRanges() == 0)
+                return;
+        }
+
+        if (options.quick)
+            return;
+
+        if (verifyDigest() && !options.extendedVerification)
+            return;
+
+        verifySSTable();
+
+        outputHandler.output("Verify of %s succeeded. All %d rows read successfully", sstable, goodRows);
+    }
+
+    private void verifySSTable()
+    {
+        long rowStart;
+        outputHandler.output("Extended Verify requested, proceeding to inspect values");
+
+        try (VerifyController verifyController = new VerifyController(cfs);
+             KeyReader indexIterator = sstable.keyReader())
+        {
+            if (indexIterator.dataPosition() != 0)
+                markAndThrow(new RuntimeException("First row position from index != 0: " + indexIterator.dataPosition()));
+
+            List<Range<Token>> ownedRanges = isOffline ? Collections.emptyList() : Range.normalize(tokenLookup.apply(cfs.metadata().keyspace));
+            RangeOwnHelper rangeOwnHelper = new RangeOwnHelper(ownedRanges);
+            DecoratedKey prevKey = null;
+
+            while (!dataFile.isEOF())
+            {
+
+                if (verifyInfo.isStopRequested())
+                    throw new CompactionInterruptedException(verifyInfo.getCompactionInfo());
+
+                rowStart = dataFile.getFilePointer();
+                outputHandler.debug("Reading row at %d", rowStart);
+
+                DecoratedKey key = null;
+                try
+                {
+                    key = sstable.decorateKey(ByteBufferUtil.readWithShortLength(dataFile));
+                }
+                catch (Throwable th)
+                {
+                    throwIfFatal(th);

Review Comment:
   Won't we ignore things like NPEs here? Would it make sense to catch and re-throw any `RuntimeException` before we hit this fallback?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167172194


##########
src/java/org/apache/cassandra/io/sstable/format/bti/PartitionIndexBuilder.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.util.function.Consumer;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.tries.IncrementalTrieWriter;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+/**
+ * Partition index builder: stores index or data positions in an incrementally built, page aware on-disk trie.
+ *

Review Comment:
   ```suggestion
    * <p>
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1167194629


##########
src/java/org/apache/cassandra/io/tries/ReverseValueIterator.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.tries;
+
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+/**
+ * Thread-unsafe reverse value iterator for on-disk tries. Uses the assumptions of Walker.
+ */
+public class ReverseValueIterator<Concrete extends ReverseValueIterator<Concrete>> extends Walker<Concrete>
+{
+    private final ByteSource limit;
+    private IterationPosition stack;
+    private long next;
+    private boolean reportingPrefixes;
+
+    static class IterationPosition
+    {
+        long node;
+        int childIndex;
+        int limit;
+        IterationPosition prev;

Review Comment:
   nit: `prev` can be final



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1185509431


##########
test/unit/org/apache/cassandra/io/sstable/format/bti/PartitionIndexTest.java:
##########
@@ -0,0 +1,934 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Supplier;
+
+import com.google.common.collect.HashMultiset;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multiset;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.ChunkCache;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.tries.TrieNode;
+import org.apache.cassandra.io.tries.Walker;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.PageAware;
+import org.apache.cassandra.io.util.Rebufferer;
+import org.apache.cassandra.io.util.SequentialWriter;
+import org.apache.cassandra.io.util.SequentialWriterOption;
+import org.apache.cassandra.io.util.WrappingRebufferer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@RunWith(Parameterized.class)
+public class PartitionIndexTest
+{
+    private final static Logger logger = LoggerFactory.getLogger(PartitionIndexTest.class);
+
+    private final static long SEED = System.nanoTime();
+    private final static Random random = new Random(SEED);
+
+    static final ByteComparable.Version VERSION = Walker.BYTE_COMPARABLE_VERSION;
+
+    static
+    {
+        DatabaseDescriptor.daemonInitialization();
+    }
+
+    IPartitioner partitioner = Util.testPartitioner();
+    //Lower the size of the indexes when running without the chunk cache, otherwise the test times out on Jenkins
+    static final int COUNT = ChunkCache.instance != null ? 245256 : 24525;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> generateData()
+    {
+        return Arrays.asList(new Object[]{ Config.DiskAccessMode.standard },
+                             new Object[]{ Config.DiskAccessMode.mmap });
+    }
+
+    @Parameterized.Parameter(value = 0)
+    public static Config.DiskAccessMode accessMode = Config.DiskAccessMode.standard;
+
+    public static void beforeClass()
+    {
+        logger.info("Using random seed: {}", SEED);
+    }
+
+    /**
+     * Tests last-nodes-sizing failure uncovered during code review.
+     */
+    @Test
+    public void testSizingBug() throws IOException, InterruptedException
+    {
+        for (int i = 1; i < COUNT; i *= 10)
+        {
+            testGetEq(generateRandomIndex(i));
+            testGetEq(generateSequentialIndex(i));
+        }
+    }
+
+    @Test
+    public void testGetEq() throws IOException, InterruptedException
+    {
+        testGetEq(generateRandomIndex(COUNT));
+        testGetEq(generateSequentialIndex(COUNT));
+    }
+
+    @Test
+    public void testBrokenFile() throws IOException, InterruptedException
+    {
+        // put some garbage in the file
+        final Pair<List<DecoratedKey>, PartitionIndex> data = generateRandomIndex(COUNT);
+        File f = new File(data.right.getFileHandle().path());
+        try (FileChannel ch = FileChannel.open(f.toPath(), StandardOpenOption.WRITE))
+        {
+            ch.write(generateRandomKey().getKey(), f.length() * 2 / 3);
+        }
+
+        boolean thrown = false;
+        try
+        {
+            testGetEq(data);
+        }
+        catch (Throwable e)
+        {
+            thrown = true;
+        }
+        assertTrue(thrown);
+    }
+
+    @Test
+    public void testLongKeys() throws IOException, InterruptedException
+    {
+        testGetEq(generateLongKeysIndex(COUNT / 10));
+    }
+
+    void testGetEq(Pair<List<DecoratedKey>, PartitionIndex> data)
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, reader.exactCandidate(keys.get(i)));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(eq(keys, key), eq(keys, key, reader.exactCandidate(key)));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGt() throws IOException
+    {
+        testGetGt(generateRandomIndex(COUNT));
+        testGetGt(generateSequentialIndex(COUNT));
+    }
+
+    private void testGetGt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i < data.left.size() - 1 ? i + 1 : -1, gt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(gt(keys, key), gt(keys, key, reader));
+            }
+        }
+    }
+
+    @Test
+    public void testGetGe() throws IOException
+    {
+        testGetGe(generateRandomIndex(COUNT));
+        testGetGe(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetGe(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i, ge(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(ge(keys, key), ge(keys, key, reader));
+            }
+        }
+    }
+
+
+    @Test
+    public void testGetLt() throws IOException
+    {
+        testGetLt(generateRandomIndex(COUNT));
+        testGetLt(generateSequentialIndex(COUNT));
+    }
+
+    public void testGetLt(Pair<List<DecoratedKey>, PartitionIndex> data) throws IOException
+    {
+        List<DecoratedKey> keys = data.left;
+        try (PartitionIndex summary = data.right;
+             PartitionIndex.Reader reader = summary.openReader())
+        {
+            for (int i = 0; i < data.left.size(); i++)
+            {
+                assertEquals(i - 1, lt(keys, keys.get(i), reader));
+                DecoratedKey key = generateRandomKey();
+                assertEquals(lt(keys, key), lt(keys, key, reader));
+            }
+        }
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) > 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.ceiling(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) >= 0) ? pos : null)).orElse(-1L);
+    }
+
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key, PartitionIndex.Reader summary) throws IOException
+    {
+        return Optional.ofNullable(summary.floor(key, (pos, assumeNoMatch, sk) -> (assumeNoMatch || keys.get((int) pos).compareTo(sk) < 0) ? pos : null)).orElse(-1L);
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key, long exactCandidate)
+    {
+        int idx = (int) exactCandidate;
+        if (exactCandidate == PartitionIndex.NOT_FOUND)
+            return -1;
+        return (keys.get(idx).equals(key)) ? idx : -1;
+    }
+
+    private long gt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        else
+            ++index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long lt(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+
+        if (index < 0)
+            index = -index - 2;
+
+        return index >= 0 ? index : -1;
+    }
+
+    private long ge(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        if (index < 0)
+            index = -1 - index;
+        return index < keys.size() ? index : -1;
+    }
+
+    private long eq(List<DecoratedKey> keys, DecoratedKey key)
+    {
+        int index = Collections.binarySearch(keys, key);
+        return index >= 0 ? index : -1;
+    }
+
+    @Test
+    public void testAddEmptyKey() throws Exception
+    {
+        IPartitioner p = new RandomPartitioner();
+        File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+
+        FileHandle.Builder fhBuilder = makeHandle(file);
+        try (SequentialWriter writer = makeWriter(file);
+             PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+        )
+        {
+            DecoratedKey key = p.decorateKey(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            builder.addEntry(key, 42);
+            builder.complete();
+            try (PartitionIndex summary = loadPartitionIndex(fhBuilder, writer);
+                 PartitionIndex.Reader reader = summary.openReader())
+            {
+                assertEquals(1, summary.size());
+                assertEquals(42, reader.getLastIndexPosition());
+                assertEquals(42, reader.exactCandidate(key));
+            }
+        }
+    }
+
+    @Test
+    public void testIteration() throws IOException
+    {
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        checkIteration(random.left, random.left.size(), random.right);
+        random.right.close();
+//        assertEquals(0, ChunkReader.bufferPool.usedMemoryBytes());
+    }
+
+    @Test
+    public void testZeroCopyOffsets() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndexWithZeroCopy(COUNT, 1, COUNT - 2);
+        List<DecoratedKey> keys = random.left;
+        try (PartitionIndex index = random.right)
+        {
+            assertEquals(COUNT - 2, index.size());
+            assertEquals(keys.get(1), index.firstKey());
+            assertEquals(keys.get(COUNT - 2), index.lastKey());
+        }
+    }
+
+    public void checkIteration(List<DecoratedKey> keys, int keysSize, PartitionIndex index)
+    {
+        try (PartitionIndex enforceIndexClosing = index;
+             PartitionIndex.IndexPosIterator iter = index.allKeysIterator())
+        {
+            int i = 0;
+            while (true)
+            {
+                long pos = iter.nextIndexPos();
+                if (pos == PartitionIndex.NOT_FOUND)
+                    break;
+                assertEquals(i, pos);
+                ++i;
+            }
+            assertEquals(keysSize, i);
+        }
+    }
+
+    @Test
+    public void testConstrainedIteration() throws IOException
+    {
+        Pair<List<DecoratedKey>, PartitionIndex> random = generateRandomIndex(COUNT);
+        try (PartitionIndex summary = random.right)
+        {
+            List<DecoratedKey> keys = random.left;
+            Random rand = new Random();
+
+            for (int i = 0; i < 1000; ++i)
+            {
+                boolean exactLeft = rand.nextBoolean();
+                boolean exactRight = rand.nextBoolean();
+                DecoratedKey left = exactLeft ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                DecoratedKey right = exactRight ? keys.get(rand.nextInt(keys.size())) : generateRandomKey();
+                if (right.compareTo(left) < 0)
+                {
+                    DecoratedKey t = left;
+                    left = right;
+                    right = t;
+                    boolean b = exactLeft;
+                    exactLeft = exactRight;
+                    exactRight = b;
+                }
+
+                try (PartitionIndex.IndexPosIterator iter = new PartitionIndex.IndexPosIterator(summary, left, right))
+                {
+                    long p = iter.nextIndexPos();
+                    if (p == PartitionIndex.NOT_FOUND)
+                    {
+                        int idx = (int) ge(keys, left); // first greater key
+                        if (idx == -1)
+                            continue;
+                        assertTrue(left + " <= " + keys.get(idx) + " <= " + right + " but " + idx + " wasn't iterated.", right.compareTo(keys.get(idx)) < 0);
+                        continue;
+                    }
+
+                    int idx = (int) p;
+                    if (p > 0)
+                        assertTrue(left.compareTo(keys.get(idx - 1)) > 0);
+                    if (p < keys.size() - 1)
+                        assertTrue(left.compareTo(keys.get(idx + 1)) < 0);
+                    if (exactLeft)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(left == keys.get(idx));
+                    while (true)
+                    {
+                        ++idx;
+                        long pos = iter.nextIndexPos();
+                        if (pos == PartitionIndex.NOT_FOUND)
+                            break;
+                        assertEquals(idx, pos);
+                    }
+                    --idx; // seek at last returned
+                    if (idx < keys.size() - 1)
+                        assertTrue(right.compareTo(keys.get(idx + 1)) < 0);
+                    if (idx > 0)
+                        assertTrue(right.compareTo(keys.get(idx - 1)) > 0);
+                    if (exactRight)      // must be precise on exact, otherwise could be in any relation
+                        assertTrue(right == keys.get(idx));
+                }
+                catch (AssertionError e)
+                {
+                    StringBuilder buf = new StringBuilder();
+                    buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    try (PartitionIndex.IndexPosIterator iter2 = new PartitionIndex.IndexPosIterator(summary, left, right))
+                    {
+                        long pos;
+                        while ((pos = iter2.nextIndexPos()) != PartitionIndex.NOT_FOUND)
+                            buf.append(keys.get((int) pos).byteComparableAsString(VERSION)).append("\n");
+                        buf.append(String.format("Left %s%s Right %s%s%n", left.byteComparableAsString(VERSION), exactLeft ? "#" : "", right.byteComparableAsString(VERSION), exactRight ? "#" : ""));
+                    }
+                    logger.error(buf.toString(), e);
+                    throw e;
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testPartialIndex() throws IOException
+    {
+        for (int reps = 0; reps < 10; ++reps)
+        {
+            File file = FileUtils.createTempFile("ColumnTrieReaderTest", "");
+            List<DecoratedKey> list = Lists.newArrayList();
+            int parts = 15;
+            FileHandle.Builder fhBuilder = makeHandle(file);
+            try (SequentialWriter writer = makeWriter(file);
+                 PartitionIndexBuilder builder = new PartitionIndexBuilder(writer, fhBuilder)
+            )
+            {
+                writer.setPostFlushListener(() -> builder.markPartitionIndexSynced(writer.getLastFlushOffset()));
+                for (int i = 0; i < COUNT; i++)
+                {
+                    DecoratedKey key = generateRandomLengthKey();
+                    list.add(key);
+                }
+                Collections.sort(list);
+                AtomicInteger callCount = new AtomicInteger();
+
+                int i = 0;
+                for (int part = 1; part <= parts; ++part)
+                {
+                    for (; i < COUNT * part / parts; i++)
+                        builder.addEntry(list.get(i), i);
+
+                    final long addedSize = i;
+                    builder.buildPartial(index ->
+                                         {
+                                             int indexSize = Collections.binarySearch(list, index.lastKey()) + 1;
+                                             assert indexSize >= addedSize - 1;
+                                             checkIteration(list, indexSize, index);
+                                             callCount.incrementAndGet();
+                                         }, 0, i * 1024L);
+                    builder.markDataSynced(i * 1024L);
+                    // verifier will be called when the sequentialWriter finishes a chunk
+                }
+
+                for (; i < COUNT; ++i)
+                    builder.addEntry(list.get(i), i);
+                builder.complete();
+                try (PartitionIndex index = loadPartitionIndex(fhBuilder, writer))
+                {
+                    checkIteration(list, list.size(), index);
+                }
+                if (COUNT / parts > 16000)
+                {
+                    assertTrue(String.format("Expected %d or %d calls, got %d", parts, parts - 1, callCount.get()),
+                               callCount.get() == parts - 1 || callCount.get() == parts);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Test
+    public void testDeepRecursion() throws InterruptedException

Review Comment:
   nit: unnecessary `throws`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1181911804


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableReader.java:
##########
@@ -0,0 +1,558 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DataRange;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IVerifier;
+import org.apache.cassandra.io.sstable.SSTable;
+import org.apache.cassandra.io.sstable.SSTableReadsListener;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SelectionReason;
+import org.apache.cassandra.io.sstable.SSTableReadsListener.SkippingReason;
+import org.apache.cassandra.io.sstable.format.SSTableReaderWithFilter;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.IFilter;
+import org.apache.cassandra.utils.OutputHandler;
+
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.EQ;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GE;
+import static org.apache.cassandra.io.sstable.format.SSTableReader.Operator.GT;
+import static org.apache.cassandra.utils.concurrent.SharedCloseable.sharedCopyOrNull;
+
+/**
+ * Reader of SSTable files in BTI format (see {@link BtiFormat}), written by {@link BtiTableWriter}.
+ *
+ * SSTableReaders are open()ed by Keyspace.onStart; after that they are created by SSTableWriter.renameAndOpen.
+ * Do not re-call open() on existing SSTable files; use the references kept by ColumnFamilyStore post-start instead.
+ */
+public class BtiTableReader extends SSTableReaderWithFilter
+{
+    private final FileHandle rowIndexFile;
+    private final PartitionIndex partitionIndex;
+
+    public BtiTableReader(Builder builder, SSTable.Owner owner)
+    {
+        super(builder, owner);
+        this.rowIndexFile = builder.getRowIndexFile();
+        this.partitionIndex = builder.getPartitionIndex();
+    }
+
+    protected final Builder unbuildTo(Builder builder, boolean sharedCopy)
+    {
+        Builder b = super.unbuildTo(builder, sharedCopy);
+        if (builder.getPartitionIndex() == null)
+            b.setPartitionIndex(sharedCopy ? sharedCopyOrNull(partitionIndex) : partitionIndex);
+        if (builder.getRowIndexFile() == null)
+            b.setRowIndexFile(sharedCopy ? sharedCopyOrNull(rowIndexFile) : rowIndexFile);
+
+        return b;
+    }
+
+    @Override
+    protected List<AutoCloseable> setupInstance(boolean trackHotness)
+    {
+        ArrayList<AutoCloseable> closeables = Lists.newArrayList(rowIndexFile, partitionIndex);
+        closeables.addAll(super.setupInstance(trackHotness));
+        return closeables;
+    }
+
+    /**
+     * Whether to filter out data before {@code first}. Needed for sources of data in a compaction, where the relevant
+     * output is opened early -- in this case the sstable's start is changed, but the data can still be found in the
+     * file. Range and point queries must filter it out.
+     */
+    protected boolean filterFirst()
+    {
+        return openReason == OpenReason.MOVED_START;
+    }
+
+    /**
+     * Whether to filter out data after {@code last}. Early-open sstables may contain data beyond the switch point
+     * (because an early-opened sstable is not ready until buffers have been flushed), and leaving that data visible
+     * will give a redundant copy with all associated overheads.
+     */
+    protected boolean filterLast()
+    {
+        return openReason == OpenReason.EARLY && partitionIndex instanceof PartitionIndexEarly;
+    }
+
+    public long estimatedKeys()
+    {
+        return partitionIndex == null ? 0 : partitionIndex.size();
+    }
+
+    @Override
+    protected TrieIndexEntry getRowIndexEntry(PartitionPosition key,
+                                              Operator operator,
+                                              boolean updateStats,
+                                              boolean permitMatchPastLast,
+                                              SSTableReadsListener listener)
+    {
+        assert !permitMatchPastLast;
+
+        PartitionPosition searchKey;
+        Operator searchOp;
+
+        if (operator == EQ)
+            return getExactPosition((DecoratedKey) key, listener, updateStats);
+
+        if (operator == GT || operator == GE)
+        {
+            if (filterLast() && last.compareTo(key) < 0)
+            {
+                notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, operator, updateStats);
+                return null;
+            }
+            boolean filteredLeft = (filterFirst() && first.compareTo(key) > 0);
+            searchKey = filteredLeft ? first : key;
+            searchOp = filteredLeft ? GE : operator;
+
+            try (PartitionIndex.Reader reader = partitionIndex.openReader())
+            {
+                TrieIndexEntry rie = reader.ceiling(searchKey, (pos, assumeNoMatch, compareKey) -> retrieveEntryIfAcceptable(searchOp, compareKey, pos, assumeNoMatch));
+                if (rie != null)
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, operator, updateStats, rie);
+                else
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, operator, updateStats);
+                return rie;
+            }
+            catch (IOException e)
+            {
+                markSuspect();
+                throw new CorruptSSTableException(e, rowIndexFile.path());
+            }
+        }
+
+        throw new IllegalArgumentException("Invalid op: " + operator);
+    }
+
+    /**
+     * Called by getPosition above (via Reader.ceiling/floor) to check if the position satisfies the full key constraint.
+     * This is called once if there is a prefix match (which can be in any relationship with the sought key, thus
+     * assumeNoMatch: false), and if it returns null it is called again for the closest greater position
+     * (with assumeNoMatch: true).
+     * Returns the index entry at this position, or null if the search op rejects it.
+     */
+    private TrieIndexEntry retrieveEntryIfAcceptable(Operator searchOp, PartitionPosition searchKey, long pos, boolean assumeNoMatch) throws IOException
+    {
+        if (pos >= 0)
+        {
+            try (FileDataInput in = rowIndexFile.createReader(pos))
+            {
+                if (assumeNoMatch)
+                    ByteBufferUtil.skipShortLength(in);
+                else
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+                return TrieIndexEntry.deserialize(in, in.getFilePointer());
+            }
+        }
+        else
+        {
+            pos = ~pos;
+            if (!assumeNoMatch)
+            {
+                try (FileDataInput in = dfile.createReader(pos))
+                {
+                    ByteBuffer indexKey = ByteBufferUtil.readWithShortLength(in);
+                    DecoratedKey decorated = decorateKey(indexKey);
+                    if (searchOp.apply(decorated.compareTo(searchKey)) != 0)
+                        return null;
+                }
+            }
+            return new TrieIndexEntry(pos);
+        }
+    }
+
+    @Override
+    public DecoratedKey keyAtPositionFromSecondaryIndex(long keyPositionFromSecondaryIndex) throws IOException
+    {
+        try (RandomAccessReader reader = openDataReader())
+        {
+            reader.seek(keyPositionFromSecondaryIndex);
+            if (reader.isEOF())
+                return null;
+            return decorateKey(ByteBufferUtil.readWithShortLength(reader));
+        }
+    }
+
+    public TrieIndexEntry getExactPosition(DecoratedKey dk,
+                                           SSTableReadsListener listener,
+                                           boolean updateStats)
+    {
+        if ((filterFirst() && first.compareTo(dk) > 0) || (filterLast() && last.compareTo(dk) < 0))
+        {
+            notifySkipped(SkippingReason.MIN_MAX_KEYS, listener, EQ, updateStats);
+            return null;
+        }
+
+        if (!isPresentInFilter(dk))
+        {
+            notifySkipped(SkippingReason.BLOOM_FILTER, listener, EQ, updateStats);
+            return null;
+        }
+
+        try (PartitionIndex.Reader reader = partitionIndex.openReader())
+        {
+            long indexPos = reader.exactCandidate(dk);
+            if (indexPos == PartitionIndex.NOT_FOUND)
+            {
+                notifySkipped(SkippingReason.PARTITION_INDEX_LOOKUP, listener, EQ, updateStats);
+                return null;
+            }
+
+            FileHandle fh;
+            long seekPosition;
+            if (indexPos >= 0)
+            {
+                fh = rowIndexFile;
+                seekPosition = indexPos;
+            }
+            else
+            {
+                fh = dfile;
+                seekPosition = ~indexPos;
+            }
+
+            try (FileDataInput in = fh.createReader(seekPosition))
+            {
+                if (ByteBufferUtil.equalsWithShortLength(in, dk.getKey()))
+                {
+                    TrieIndexEntry rie = indexPos >= 0 ? TrieIndexEntry.deserialize(in, in.getFilePointer())
+                                                       : new TrieIndexEntry(~indexPos);
+                    notifySelected(SelectionReason.INDEX_ENTRY_FOUND, listener, EQ, updateStats, rie);
+                    return rie;
+                }
+                else
+                {
+                    notifySkipped(SkippingReason.INDEX_ENTRY_NOT_FOUND, listener, EQ, updateStats);
+                    return null;
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            markSuspect();
+            throw new CorruptSSTableException(e, rowIndexFile.path());
+        }
+    }
+
+    /**
+     * @param bounds Must not be wrapped around ranges
+     * @return PartitionIndexIterator within the given bounds
+     */
+    public PartitionIterator coveredKeysIterator(AbstractBounds<PartitionPosition> bounds) throws IOException
+    {
+        return coveredKeysIterator(bounds.left, bounds.inclusiveLeft(), bounds.right, bounds.inclusiveRight());
+    }
+
+    public ScrubPartitionIterator scrubPartitionsIterator() throws IOException
+    {
+        return new ScrubIterator(partitionIndex, rowIndexFile);
+    }
+
+    public PartitionIterator coveredKeysIterator(PartitionPosition left, boolean inclusiveLeft, PartitionPosition right, boolean inclusiveRight) throws IOException
+    {
+        if (filterFirst() && left.compareTo(first) < 0)
+        {
+            left = first;
+            inclusiveLeft = true;
+        }
+        if (filterLast() && right.compareTo(last) > 0)
+        {
+            right = last;
+            inclusiveRight = true;
+        }
+        // If a bound was adjusted, also check that the resulting bounds did not become empty.
+        if (filterFirst() || filterLast())
+        {
+            int cmp = left.compareTo(right);
+            if (cmp > 0 || cmp == 0 && !(inclusiveLeft && inclusiveRight))
+                return PartitionIterator.empty(partitionIndex);
+        }
+
+        return PartitionIterator.create(partitionIndex,
+                                        metadata().partitioner,
+                                        rowIndexFile,
+                                        dfile,
+                                        left, inclusiveLeft ? -1 : 0,
+                                        right, inclusiveRight ? 0 : -1);
+    }
+
+    @Override
+    public PartitionIterator keyReader() throws IOException
+    {
+        return PartitionIterator.create(partitionIndex, metadata().partitioner, rowIndexFile, dfile);
+    }
+
+    @Override
+    public Iterable<DecoratedKey> getKeySamples(final Range<Token> range)
+    {
+        // BTI does not support key sampling as it would involve walking the index or data file.
+        // Validator has an alternate solution for empty key sample lists.
+        return Collections.emptyList();
+    }
+
+    @Override
+    public long estimatedKeysForRanges(Collection<Range<Token>> ranges)
+    {
+        // Estimate the number of partitions by calculating the bytes of the sstable that are covered by the specified
+        // ranges and using the mean partition size to obtain a number of partitions from that.
+        long selectedDataSize = 0;
+        for (Range<Token> range : Range.normalize(ranges))
+        {
+            PartitionPosition left = range.left.minKeyBound();
+            if (left.compareTo(first) <= 0)
+                left = null;
+            else if (left.compareTo(last) > 0)
+                continue;   // no intersection
+
+            PartitionPosition right = range.right.minKeyBound();
+            if (range.right.isMinimum() || right.compareTo(last) >= 0)
+                right = null;
+            else if (right.compareTo(first) < 0)
+                continue;   // no intersection
+
+            if (left == null && right == null)
+                return partitionIndex.size();   // sstable is fully covered, return full partition count to avoid rounding errors
+
+            if (left == null && filterFirst())
+                left = first;
+            if (right == null && filterLast())
+                right = last;
+
+            long startPos = left != null ? getPosition(left, GE) : 0;
+            long endPos = right != null ? getPosition(right, GE) : uncompressedLength();
+            selectedDataSize += endPos - startPos;
+        }
+        return Math.round(selectedDataSize / sstableMetadata.estimatedPartitionSize.rawMean());
+    }
+
+
+    @Override
+    public UnfilteredRowIterator rowIterator(DecoratedKey key,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed,
+                                             SSTableReadsListener listener)
+    {
+        return rowIterator(null, key, getExactPosition(key, listener, true), slices, selectedColumns, reversed);
+    }
+
+    public UnfilteredRowIterator rowIterator(FileDataInput dataFileInput,
+                                             DecoratedKey key,
+                                             TrieIndexEntry indexEntry,
+                                             Slices slices,
+                                             ColumnFilter selectedColumns,
+                                             boolean reversed)
+    {
+        if (indexEntry == null)
+            return UnfilteredRowIterators.noRowsIterator(metadata(), key, Rows.EMPTY_STATIC_ROW, DeletionTime.LIVE, reversed);
+
+        if (reversed)
+            return new SSTableReversedIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+        else
+            return new SSTableIterator(this, dataFileInput, key, indexEntry, slices, selectedColumns, rowIndexFile);
+    }
+
+    @Override
+    public ISSTableScanner getScanner()
+    {
+        return BtiTableScanner.getScanner(this);
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Collection<Range<Token>> ranges)
+    {
+        if (ranges != null)
+            return BtiTableScanner.getScanner(this, ranges);
+        else
+            return getScanner();
+    }
+
+    @Override
+    public ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
+    {
+        return BtiTableScanner.getScanner(this, rangeIterator);
+    }
+
+    @VisibleForTesting
+    @Override
+    public BtiTableReader cloneAndReplace(IFilter filter)
+    {
+        return unbuildTo(new Builder(descriptor).setFilter(filter), true).build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public BtiTableReader cloneWithRestoredStart(DecoratedKey restoredStart)
+    {
+        return runWithLock(ignored -> cloneAndReplace(restoredStart, OpenReason.NORMAL));
+    }
+
+    @Override
+    public BtiTableReader cloneWithNewStart(DecoratedKey newStart)
+    {
+        return runWithLock(d -> {
+            assert openReason != OpenReason.EARLY;
+            // TODO: merge with caller's firstKeyBeyond() work,to save time
+            if (newStart.compareTo(first) > 0)
+            {
+                final long dataStart = getPosition(newStart, Operator.EQ);
+                runOnClose(() -> dfile.dropPageCache(dataStart));
+            }
+
+            return cloneAndReplace(newStart, OpenReason.MOVED_START);
+        });
+    }
+
+    /**
+     * Clone this reader with the provided start and open reason, and set the clone as replacement.
+     *
+     * @param newFirst the first key for the replacement (which can be different from the original due to the pre-emptive
+     *                 opening of compaction results).
+     * @param reason   the {@code OpenReason} for the replacement.
+     * @return the cloned reader. That reader is set as a replacement by the method.
+     */
+    private BtiTableReader cloneAndReplace(DecoratedKey newFirst, OpenReason reason)
+    {
+        return unbuildTo(new Builder(descriptor), true)
+               .setFirst(newFirst)
+               .setOpenReason(reason)
+               .build(owner().orElse(null), true, true);
+    }
+
+    @Override
+    public DecoratedKey firstKeyBeyond(PartitionPosition token)
+    {
+        try
+        {
+            TrieIndexEntry pos = getRowIndexEntry(token, Operator.GT, true, false, SSTableReadsListener.NOOP_LISTENER);

Review Comment:
   Just to confirm my understanding, `getRowIndexEntry()` produces a data file position in `pos.position` whether the partition is indexed or not?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2267: CASSANDRA-18398: Trie-indexed SSTable format

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2267:
URL: https://github.com/apache/cassandra/pull/2267#discussion_r1183030818


##########
src/java/org/apache/cassandra/io/sstable/format/bti/BtiTableScrubber.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.sstable.format.bti;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.compaction.CompactionInterruptedException;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.IScrubber;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.format.SortedTableScrubber;
+import org.apache.cassandra.io.sstable.format.bti.BtiFormat.Components;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Throwables;
+
+public class BtiTableScrubber extends SortedTableScrubber<BtiTableReader> implements IScrubber
+{
+    private final boolean isIndex;
+    private ScrubPartitionIterator indexIterator;
+
+    public BtiTableScrubber(ColumnFamilyStore cfs,
+                            LifecycleTransaction transaction,
+                            OutputHandler outputHandler,
+                            IScrubber.Options options)
+    {
+        super(cfs, transaction, outputHandler, options);
+
+        boolean hasIndexFile = sstable.getComponents().contains(Components.PARTITION_INDEX);
+        this.isIndex = cfs.isIndex();
+        if (!hasIndexFile)
+        {
+            // if there's any corruption in the -Data.db then partitions can't be skipped over. but it's worth a shot.
+            outputHandler.warn("Missing index component");
+        }
+
+        try
+        {
+            this.indexIterator = hasIndexFile
+                                 ? openIndexIterator()
+                                 : null;
+        }
+        catch (RuntimeException ex)
+        {
+            outputHandler.warn("Detected corruption in the index file - cannot open index iterator", ex);
+        }
+    }
+
+    private ScrubPartitionIterator openIndexIterator()
+    {
+        try
+        {
+            return sstable.scrubPartitionsIterator();
+        }
+        catch (Throwable t)
+        {
+            outputHandler.warn(t, "Index is unreadable, scrubbing will continue without index.");
+        }
+        return null;
+    }
+
+    @Override
+    protected UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
+    {
+        return options.checkData && !isIndex ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
+    }
+
+    public void scrubInternal(SSTableRewriter writer)
+    {
+        assert !indexAvailable() || indexIterator.dataPosition() == 0 : indexIterator.dataPosition();
+
+        DecoratedKey prevKey = null;
+
+        while (!dataFile.isEOF())
+        {
+            if (scrubInfo.isStopRequested())
+                throw new CompactionInterruptedException(scrubInfo.getCompactionInfo());
+
+            // position in a data file where the partition starts
+            long dataStart = dataFile.getFilePointer();
+            outputHandler.debug("Reading row at %d", dataStart);
+
+            DecoratedKey key = null;
+            Throwable keyReadError = null;
+            try
+            {
+                ByteBuffer raw = ByteBufferUtil.readWithShortLength(dataFile);
+                if (!cfs.metadata.getLocal().isIndex())

Review Comment:
   nit: Do we have to check this on every iteration? (i.e. Is the table going to change type in the middle of this loop?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org