diff --git a/CHANGES.txt b/CHANGES.txt index 443f844d4b5a..d114f3028c62 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ Future version (tbd) + * Reduce size of per-SSTable index components for SAI (CASSANDRA-18673) * Require only MODIFY permission on base when updating table with MV (STAR-564) Merged from 5.1: * Expose current compaction throughput in nodetool (CASSANDRA-13890) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 436f8448332c..73ff47dc8b75 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -392,6 +392,16 @@ public enum CassandraRelevantProperties /** Controls the maximum number of expressions that will be used in a SAI intersection operation. */ SAI_INTERSECTION_CLAUSE_LIMIT("cassandra.sai.intersection.clause.limit", "2"), + /** + * Used to determine the block size and block mask for the clustering sorted terms. + */ + SAI_SORTED_TERMS_CLUSTERING_BLOCK_SHIFT("cassandra.sai.sorted_terms_clustering_block_shift", "4"), + + /** + * Used to determine the block size and block mask for the partition sorted terms. + */ + SAI_SORTED_TERMS_PARTITION_BLOCK_SHIFT("cassandra.sai.sorted_terms_partition_block_shift", "4"), + /** Whether vector type only allows float vectors. True by default. **/ VECTOR_FLOAT_ONLY("cassandra.float_only_vectors", "true"), /** Enables use of vector type. True by default. **/ diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java index e24f766fd288..1d0e61825c52 100644 --- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java +++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java @@ -33,7 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.cql3.AssignmentTestable; import org.apache.cassandra.cql3.CQL3Type; import org.apache.cassandra.cql3.ColumnSpecification; @@ -178,6 +177,11 @@ public T compose(V value, ValueAccessor accessor) return getSerializer().deserialize(value, accessor); } + public ByteBuffer decomposeUntyped(Object value) + { + return decompose((T) value); + } + public ByteBuffer decompose(T value) { return getSerializer().serialize(value); diff --git a/src/java/org/apache/cassandra/index/sai/IndexContext.java b/src/java/org/apache/cassandra/index/sai/IndexContext.java index f6eb4a974f94..2652ab753918 100644 --- a/src/java/org/apache/cassandra/index/sai/IndexContext.java +++ b/src/java/org/apache/cassandra/index/sai/IndexContext.java @@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory; import io.github.jbellis.jvector.vector.VectorSimilarityFunction; -import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.cql3.Operator; import org.apache.cassandra.cql3.statements.schema.IndexTarget; import org.apache.cassandra.db.ClusteringComparator; @@ -73,7 +72,6 @@ import org.apache.cassandra.index.sai.iterators.KeyRangeUnionIterator; import org.apache.cassandra.index.sai.memory.MemtableIndex; import org.apache.cassandra.index.sai.memory.MemtableKeyRangeIterator; -import org.apache.cassandra.index.sai.memory.TrieMemtableIndex; import org.apache.cassandra.index.sai.metrics.ColumnQueryMetrics; import org.apache.cassandra.index.sai.metrics.IndexMetrics; import org.apache.cassandra.index.sai.plan.Expression; @@ -610,7 +608,7 @@ public int getIntOption(String name, int defaultValue) } catch (NumberFormatException e) { - logger.error("Failed to parse index configuration " + name + " = " + value + " as integer"); + logger.error("Failed to parse index configuration {} = {} as integer", name, value); return defaultValue; } } @@ -998,7 +996,7 @@ public long indexFileCacheSize() public IndexFeatureSet indexFeatureSet() { IndexFeatureSet.Accumulator accumulator = new IndexFeatureSet.Accumulator(); - getView().getIndexes().stream().map(SSTableIndex::indexFeatureSet).forEach(set -> accumulator.accumulate(set)); + getView().getIndexes().stream().map(SSTableIndex::indexFeatureSet).forEach(accumulator::accumulate); return accumulator.complete(); } } diff --git a/src/java/org/apache/cassandra/index/sai/SSTableContext.java b/src/java/org/apache/cassandra/index/sai/SSTableContext.java index e18a697dcddd..9f04aa42883c 100644 --- a/src/java/org/apache/cassandra/index/sai/SSTableContext.java +++ b/src/java/org/apache/cassandra/index/sai/SSTableContext.java @@ -65,7 +65,6 @@ private SSTableContext(SSTableContext copy) this.primaryKeyMapFactory = copy.primaryKeyMapFactory; } - @SuppressWarnings("resource") public static SSTableContext create(SSTableReader sstable, IndexComponents.ForRead perSSTableComponents) { var onDiskFormat = perSSTableComponents.onDiskFormat(); @@ -146,7 +145,7 @@ public PrimaryKeyMap.Factory primaryKeyMapFactory() */ public int openFilesPerSSTable() { - return perSSTableComponents.onDiskFormat().openFilesPerSSTable(); + return perSSTableComponents.onDiskFormat().openFilesPerSSTable(sstable.hasClustering); } @Override diff --git a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java index f3c2fa362d27..ff8f2f04f20b 100644 --- a/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java +++ b/src/java/org/apache/cassandra/index/sai/StorageAttachedIndexGroup.java @@ -17,7 +17,6 @@ */ package org.apache.cassandra.index.sai; -import java.io.IOException; import java.lang.invoke.MethodHandles; import java.util.ArrayList; import java.util.Collection; @@ -60,7 +59,6 @@ import org.apache.cassandra.index.sai.plan.StorageAttachedIndexQueryPlan; import org.apache.cassandra.index.transactions.IndexTransaction; import org.apache.cassandra.io.sstable.Component; -import org.apache.cassandra.io.sstable.CorruptSSTableException; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SSTableWatcher; import org.apache.cassandra.io.sstable.format.SSTableFlushObserver; @@ -245,7 +243,7 @@ public void rangeTombstone(RangeTombstone tombstone) private void forEach(Consumer action) { - indexers.forEach(action::accept); + indexers.forEach(action); } }; } @@ -259,7 +257,7 @@ public StorageAttachedIndexQueryPlan queryPlanFor(RowFilter rowFilter) @Override public SSTableFlushObserver getFlushObserver(Descriptor descriptor, LifecycleNewTracker tracker, TableMetadata tableMetadata, long keyCount) { - IndexDescriptor indexDescriptor = IndexDescriptor.empty(descriptor); + IndexDescriptor indexDescriptor = IndexDescriptor.empty(descriptor, tableMetadata); try { return new StorageAttachedIndexWriter(indexDescriptor, tableMetadata, indices, tracker, keyCount, baseCfs.metric); @@ -284,7 +282,7 @@ public boolean handles(IndexTransaction.Type type) @Override public Set componentsForNewSSTable() { - return IndexDescriptor.componentsForNewlyFlushedSSTable(indices); + return IndexDescriptor.componentsForNewlyFlushedSSTable(indices, baseCfs.metadata().comparator.size() > 0); } @Override @@ -335,7 +333,7 @@ public void handleNotification(INotification notification, Object sender) // Avoid validation for index files just written following Memtable flush. ZCS streaming should // validate index checksum. - boolean validate = notice.fromStreaming() || !notice.memtable().isPresent(); + boolean validate = notice.fromStreaming() || notice.memtable().isEmpty(); onSSTableChanged(Collections.emptySet(), notice.added, indices, validate); } else if (notification instanceof SSTableListChangedNotification) @@ -435,7 +433,7 @@ public int totalIndexBuildsInProgress() */ public int totalQueryableIndexCount() { - return (int) indices.stream().filter(i -> baseCfs.indexManager.isIndexQueryable(i)).count(); + return (int) indices.stream().filter(baseCfs.indexManager::isIndexQueryable).count(); } /** @@ -512,7 +510,7 @@ public void unsafeReload() public void reset() { contextManager.clear(); - indices.forEach(index -> index.makeIndexNonQueryable()); + indices.forEach(StorageAttachedIndex::makeIndexNonQueryable); onSSTableChanged(baseCfs.getLiveSSTables(), Collections.emptySet(), indices, false); } } diff --git a/src/java/org/apache/cassandra/index/sai/disk/PerSSTableWriter.java b/src/java/org/apache/cassandra/index/sai/disk/PerSSTableWriter.java index a14357eaa3b5..18c73b3c9dfd 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/PerSSTableWriter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/PerSSTableWriter.java @@ -22,7 +22,6 @@ import com.google.common.base.Stopwatch; import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.rows.Unfiltered; import org.apache.cassandra.index.sai.utils.PrimaryKey; /** @@ -30,11 +29,14 @@ */ public interface PerSSTableWriter { - public static final PerSSTableWriter NONE = (key) -> {}; + PerSSTableWriter NONE = key -> {}; default void startPartition(long position) throws IOException {} + default void startPartition(DecoratedKey decoratedKey) throws IOException + {} + void nextRow(PrimaryKey primaryKey) throws IOException; default void complete(Stopwatch stopwatch) throws IOException diff --git a/src/java/org/apache/cassandra/index/sai/disk/PostingListKeyRangeIterator.java b/src/java/org/apache/cassandra/index/sai/disk/PostingListKeyRangeIterator.java index 3afc61f46239..a82a7e3999ca 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/PostingListKeyRangeIterator.java +++ b/src/java/org/apache/cassandra/index/sai/disk/PostingListKeyRangeIterator.java @@ -140,11 +140,11 @@ public void close() throws IOException FileUtils.closeQuietly(postingList, primaryKeyMap); } - else { + else + { logger.warn("PostingListKeyRangeIterator is already closed", new IllegalStateException("PostingListKeyRangeIterator is already closed")); } - } private boolean exhausted() @@ -160,7 +160,7 @@ private long getNextRowId() throws IOException long segmentRowId; if (needsSkipping) { - long targetSstableRowId = primaryKeyMap.ceiling(skipToToken); + long targetSstableRowId = primaryKeyMap.rowIdFromPrimaryKey(skipToToken); // skipToToken is larger than max token in token file if (targetSstableRowId < 0) { diff --git a/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMap.java index 5a0e5e712545..0505220b84d3 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMap.java +++ b/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMap.java @@ -38,7 +38,7 @@ public interface PrimaryKeyMap extends Closeable * A factory for creating {@link PrimaryKeyMap} instances. Implementations of this * interface are expected to be threadsafe. */ - public interface Factory extends Closeable + interface Factory extends Closeable { /** * Creates a new {@link PrimaryKeyMap} instance @@ -110,7 +110,7 @@ default PrimaryKey primaryKeyFromRowId(long sstableRowId, @Nonnull PrimaryKey lo * @param key the {@link PrimaryKey} to lookup * @return the row Id associated with the {@link PrimaryKey} */ - long exactRowIdOrInvertedCeiling(PrimaryKey key); + long rowIdFromPrimaryKey(PrimaryKey key); /** * Returns the sstable row id associated with the least {@link PrimaryKey} greater than or equal to the given @@ -121,7 +121,7 @@ default PrimaryKey primaryKeyFromRowId(long sstableRowId, @Nonnull PrimaryKey lo * @param key the {@link PrimaryKey} to lookup * @return an sstable row id or a negative value if no row is found */ - long ceiling(PrimaryKey key); +// long ceiling(PrimaryKey key); /** * Returns the sstable row id associated with the greatest {@link PrimaryKey} less than or equal to the given @@ -132,7 +132,7 @@ default PrimaryKey primaryKeyFromRowId(long sstableRowId, @Nonnull PrimaryKey lo * @param key the {@link PrimaryKey} to lookup * @return an sstable row id or a negative value if no row is found */ - long floor(PrimaryKey key); +// long floor(PrimaryKey key); /** * Returns the number of primary keys in the map diff --git a/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMapIterator.java b/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMapIterator.java index b8f7e87d58d3..5c0823afef45 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMapIterator.java +++ b/src/java/org/apache/cassandra/index/sai/disk/PrimaryKeyMapIterator.java @@ -95,14 +95,14 @@ public static KeyRangeIterator create(SSTableContext ctx, AbstractBounds 0) ? minKeyBound : sstableMinKey; - long startRowId = minToken.isMinimum() ? 0 : keys.ceiling(minKey); + long startRowId = minToken.isMinimum() ? 0 : keys.rowIdFromPrimaryKey(minKey); return new PrimaryKeyMapIterator(keys, sstableMinKey, sstableMaxKey, startRowId, filter); } @Override protected void performSkipTo(PrimaryKey nextKey) { - this.currentRowId = keys.ceiling(nextKey); + this.currentRowId = keys.rowIdFromPrimaryKey(nextKey); } @Override diff --git a/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java index 59dd8dba8be2..bd5ccaf049c0 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/StorageAttachedIndexWriter.java @@ -132,6 +132,7 @@ public void startPartition(DecoratedKey key, long position) try { perSSTableWriter.startPartition(position); + perSSTableWriter.startPartition(key); } catch (Throwable t) { diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java index f4e5beeaa00d..c59abde42e6c 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponent.java @@ -26,6 +26,7 @@ import org.apache.cassandra.io.sstable.Component; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.utils.Throwables; import org.apache.lucene.store.ChecksumIndexInput; public interface IndexComponent @@ -50,10 +51,10 @@ interface ForRead extends IndexComponent @Override IndexComponents.ForRead parent(); - FileHandle createFileHandle(); + FileHandle createFileHandle(Throwables.DiscreteAction cleanup); /** - * Opens a file handle for the provided index component similarly to {@link #createFileHandle()}, + * Opens a file handle for the provided index component similarly to {@link #createFileHandle(Throwables.DiscreteAction)}, * but this method shoud be called instead of the aforemented one if the access is done during index building, that is * before the full index that this is a part of has been finalized. *

diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponentType.java b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponentType.java index bf38d6b8fbd8..1795fbf9d6c8 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponentType.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponentType.java @@ -87,19 +87,37 @@ public enum IndexComponentType * * V2 */ - PRIMARY_KEY_TRIE("PrimaryKeyTrie"), + // PRIMARY_KEY_TRIE("PrimaryKeyTrie"), + /** - * Prefix-compressed blocks of primary keys used for rowId to partition key lookups - * + * An on-disk block packed index containing the starting and ending rowIds for each partition. + */ + PARTITION_SIZES("PartitionSizes"), + + /** + * Prefix-compressed blocks of partition keys used for rowId to partition key lookups + *

* V2 */ - PRIMARY_KEY_BLOCKS("PrimaryKeyBlocks"), + PARTITION_KEY_BLOCKS("PartitionKeyBlocks"), + /** - * Encoded sequence of offsets to primary key blocks - * + * Encoded sequence of offsets to partition key blocks + *

* V2 */ - PRIMARY_KEY_BLOCK_OFFSETS("PrimaryKeyBlockOffsets"), + PARTITION_KEY_BLOCK_OFFSETS("PartitionKeyBlockOffsets"), + + /** + * Prefix-compressed blocks of clustering keys used for rowId to clustering key lookups + */ + CLUSTERING_KEY_BLOCKS("ClusteringKeyBlocks"), + + /** + * Encoded sequence of offsets to clustering key blocks + */ + CLUSTERING_KEY_BLOCK_OFFSETS("ClusteringKeyBlockOffsets"), + /** * Stores per-sstable metadata. * diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponents.java b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponents.java index 2df83e4ce2a6..355756b68cf1 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponents.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/IndexComponents.java @@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory; import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.ClusteringComparator; import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.index.sai.IndexContext; import org.apache.cassandra.io.sstable.Component; @@ -182,9 +183,13 @@ default Set expectedComponentsForVersion() { return isPerIndexGroup() ? onDiskFormat().perIndexComponentTypes(context()) - : onDiskFormat().perSSTableComponentTypes(); + : onDiskFormat().perSSTableComponentTypes(hasClustering()); } + boolean hasClustering(); + + ClusteringComparator comparator(); + default ByteComparable.Version byteComparableVersionFor(IndexComponentType component) { return version().byteComparableVersionFor(component, descriptor().version); diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/IndexDescriptor.java b/src/java/org/apache/cassandra/index/sai/disk/format/IndexDescriptor.java index 5ca3578ceea0..b2c6495dc927 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/IndexDescriptor.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/IndexDescriptor.java @@ -33,12 +33,14 @@ import javax.annotation.Nullable; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.ClusteringComparator; import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.index.sai.IndexContext; import org.apache.cassandra.index.sai.StorageAttachedIndex; @@ -53,9 +55,11 @@ import org.apache.cassandra.io.storage.StorageProvider; import org.apache.cassandra.io.util.File; import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.schema.TableMetadata; import org.apache.cassandra.utils.NoSpamLogger; import org.apache.lucene.store.BufferedChecksumIndexInput; import org.apache.lucene.store.ChecksumIndexInput; +import org.apache.cassandra.utils.Throwables; import org.apache.lucene.util.IOUtils; /** @@ -86,30 +90,46 @@ public class IndexDescriptor // are per-sstable (`isRowAware`) and some are per-column (`hasTermsHistogram`). public final Descriptor descriptor; + public final boolean hasClustering; + private final ClusteringComparator comparator; // The per-sstable components for this descriptor. This is never `null` in practice, but 1) it's a bit easier to // initialize it outsides of the ctor, and 2) it can actually change upon calls to `reload`. private IndexComponentsImpl perSSTable; private final Map perIndexes = Maps.newHashMap(); - private IndexDescriptor(Descriptor descriptor) + private IndexDescriptor(Descriptor descriptor, ClusteringComparator comparator) { this.descriptor = descriptor; + this.comparator = comparator; + this.hasClustering = comparator.size() > 0; } + @VisibleForTesting public static IndexDescriptor empty(Descriptor descriptor) { - IndexDescriptor created = new IndexDescriptor(descriptor); + IndexDescriptor created = new IndexDescriptor(descriptor, new ClusteringComparator()); // Some code assumes that you can always at least call `perSSTableComponents()` and not get `null`, so we // set it to an empty group here. created.perSSTable = created.createEmptyGroup(null); return created; } + public static IndexDescriptor empty(Descriptor descriptor, TableMetadata metadata) + { + IndexDescriptor created = new IndexDescriptor(descriptor, metadata.comparator); + // Some code assumes that you can always at least call `perSSTableComponents()` and not get `null`, so we + // set it to an empty group here. + created.perSSTable = created.createEmptyGroup(null); + return created; + } + + public static IndexDescriptor load(SSTableReader sstable, Set indices) { SSTableIndexComponentsState discovered = IndexComponentDiscovery.instance().discoverComponents(sstable); - IndexDescriptor descriptor = new IndexDescriptor(sstable.descriptor); + IndexDescriptor descriptor = new IndexDescriptor(sstable.descriptor, + sstable.metadata().comparator); descriptor.initialize(indices, discovered); return descriptor; } @@ -129,7 +149,7 @@ private void initializeIndexes(Set indices, SSTableIndexComponents private Set expectedComponentsForVersion(Version version, @Nullable IndexContext context) { return context == null - ? version.onDiskFormat().perSSTableComponentTypes() + ? version.onDiskFormat().perSSTableComponentTypes(hasClustering) : version.onDiskFormat().perIndexComponentTypes(context); } @@ -177,11 +197,11 @@ private IndexComponentsImpl createEmptyGroup(@Nullable IndexContext context) * Please note that the final sstable may not contain all of these components, as some may be empty or not written * due to the specific of the flush, but this should be a superset of the components written. */ - public static Set componentsForNewlyFlushedSSTable(Collection indices) + public static Set componentsForNewlyFlushedSSTable(Collection indices, boolean hasClustering) { ComponentsBuildId buildId = ComponentsBuildId.forNewSSTable(); Set components = new HashSet<>(); - for (IndexComponentType component : buildId.version().onDiskFormat().perSSTableComponentTypes()) + for (IndexComponentType component : buildId.version().onDiskFormat().perSSTableComponentTypes(hasClustering)) components.add(customComponentFor(buildId, component, null)); for (StorageAttachedIndex index : indices) @@ -192,7 +212,7 @@ public static Set componentsForNewlyFlushedSSTable(Collection - * This is a subset of {@link #componentsForNewlyFlushedSSTable(Collection)} and has the same caveats. + * This is a subset of {@link #componentsForNewlyFlushedSSTable(Collection, boolean)} and has the same caveats. */ public static Set perIndexComponentsForNewlyFlushedSSTable(IndexContext context) { @@ -265,6 +285,62 @@ public IndexComponents.ForWrite newPerSSTableComponentsForWrite() return newComponentsForWrite(null, perSSTable); } +// public FileHandle createPerSSTableFileHandle(IndexComponent indexComponent, Throwables.DiscreteAction cleanup) +// { +// try +// { +// final File file = fileFor(indexComponent); +// +// if (logger.isTraceEnabled()) +// { +// logger.trace(logMessage("Opening {} file handle for {} ({})"), +// file, FBUtilities.prettyPrintMemory(file.length())); +// } +// +// return new FileHandle.Builder(file).mmapped(true).complete(); +// } +// catch (Throwable t) +// { +// throw handleFileHandleCleanup(t, cleanup); +// } +// } +// +// public FileHandle createPerIndexFileHandle(IndexComponent indexComponent, IndexContext indexContext, Throwables.DiscreteAction cleanup) +// { +// try +// { +// final File file = fileFor(indexComponent, indexContext); +// +// if (logger.isTraceEnabled()) +// { +// logger.trace(indexContext.logMessage("Opening file handle for {} ({})"), +// file, FBUtilities.prettyPrintMemory(file.length())); +// } +// +// return new FileHandle.Builder(file).mmapped(true).complete(); +// } +// catch (Throwable t) +// { +// throw handleFileHandleCleanup(t, cleanup); +// } +// } + + private static RuntimeException handleFileHandleCleanup(Throwable t, @Nullable Throwables.DiscreteAction cleanup) + { + if (cleanup != null) + { + try + { + cleanup.perform(); + } + catch (Exception e) + { + return Throwables.unchecked(Throwables.merge(t, e)); + } + } + return Throwables.unchecked(t); + } + public IndexComponents.ForWrite newPerIndexComponentsForWrite(IndexContext context) { return newComponentsForWrite(context, perIndexes.get(context)); @@ -398,6 +474,17 @@ public boolean isEmpty() return isComplete() && components.size() == 1; } + @Override + public boolean hasClustering() + { + return IndexDescriptor.this.hasClustering; + } + + public ClusteringComparator comparator() + { + return IndexDescriptor.this.comparator; + } + @Override public Collection all() { @@ -521,7 +608,7 @@ public void markComplete() throws IOException { addOrGet(completionMarkerComponent()).createEmpty(); sealed = true; - // Until this call, the group is not attached to the parent. This create the link. + // Until this call, the group is not attached to the parent. This creates the link. updateParentLink(this); } @@ -607,13 +694,17 @@ public File file() } @Override - public FileHandle createFileHandle() + public FileHandle createFileHandle(Throwables.DiscreteAction cleanup) { try (var builder = StorageProvider.instance.fileHandleBuilderFor(this)) { var b = builder.order(byteOrder()); return b.complete(); } + catch (Throwable t) + { + throw handleFileHandleCleanup(t, cleanup); + } } @Override @@ -628,7 +719,7 @@ public FileHandle createIndexBuildTimeFileHandle() @Override public IndexInput openInput() { - return IndexFileUtils.instance.openBlockingInput(createFileHandle()); + return IndexFileUtils.instance.openBlockingInput(createFileHandle(null)); } @Override diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/OnDiskFormat.java b/src/java/org/apache/cassandra/index/sai/disk/format/OnDiskFormat.java index 611eab5e37b5..76b3b41867c4 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/OnDiskFormat.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/OnDiskFormat.java @@ -28,6 +28,7 @@ import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.index.sai.IndexContext; import org.apache.cassandra.index.sai.SSTableContext; +import org.apache.cassandra.index.sai.SSTableIndex; import org.apache.cassandra.index.sai.StorageAttachedIndex; import org.apache.cassandra.index.sai.disk.PerIndexWriter; import org.apache.cassandra.index.sai.disk.PerSSTableWriter; @@ -73,7 +74,7 @@ public interface OnDiskFormat * * @return the index feature set */ - public IndexFeatureSet indexFeatureSet(); + IndexFeatureSet indexFeatureSet(); /** * Returns the {@link PrimaryKey.Factory} for the on-disk format @@ -81,7 +82,7 @@ public interface OnDiskFormat * @param comparator * @return the primary key factory */ - public PrimaryKey.Factory newPrimaryKeyFactory(ClusteringComparator comparator); + PrimaryKey.Factory newPrimaryKeyFactory(ClusteringComparator comparator); /** * Returns a {@link PrimaryKeyMap.Factory} for the SSTable @@ -92,10 +93,10 @@ public interface OnDiskFormat * @return a {@link PrimaryKeyMap.Factory} for the SSTable * @throws IOException */ - public PrimaryKeyMap.Factory newPrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, PrimaryKey.Factory primaryKeyFactory, SSTableReader sstable) throws IOException; + PrimaryKeyMap.Factory newPrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, PrimaryKey.Factory primaryKeyFactory, SSTableReader sstable) throws IOException; /** - * Create a new {@link SearchableIndex} for an on-disk index. This is held by the {@SSTableIndex} + * Create a new {@link SearchableIndex} for an on-disk index. This is held by the {@link SSTableIndex} * and shared between queries. * * @param sstableContext The {@link SSTableContext} holding the per-SSTable information for the index @@ -103,7 +104,7 @@ public interface OnDiskFormat * also link to the underlying {@link IndexContext} for the index). * @return the created {@link SearchableIndex}. */ - public SearchableIndex newSearchableIndex(SSTableContext sstableContext, IndexComponents.ForRead perIndexComponents); + SearchableIndex newSearchableIndex(SSTableContext sstableContext, IndexComponents.ForRead perIndexComponents); IndexSearcher newIndexSearcher(SSTableContext sstableContext, IndexContext indexContext, @@ -117,7 +118,7 @@ IndexSearcher newIndexSearcher(SSTableContext sstableContext, * @return The {@link PerSSTableWriter} to write the per-SSTable on-disk components * @throws IOException */ - public PerSSTableWriter newPerSSTableWriter(IndexDescriptor indexDescriptor) throws IOException; + PerSSTableWriter newPerSSTableWriter(IndexDescriptor indexDescriptor) throws IOException; /** * Create a new writer for the per-index on-disk components of an index. The {@link LifecycleNewTracker} @@ -132,10 +133,10 @@ IndexSearcher newIndexSearcher(SSTableContext sstableContext, * @param keyCount * @return The {@link PerIndexWriter} that will write the per-index on-disk components */ - public PerIndexWriter newPerIndexWriter(StorageAttachedIndex index, - IndexDescriptor indexDescriptor, - LifecycleNewTracker tracker, - RowMapping rowMapping, long keyCount); + PerIndexWriter newPerIndexWriter(StorageAttachedIndex index, + IndexDescriptor indexDescriptor, + LifecycleNewTracker tracker, + RowMapping rowMapping, long keyCount); /** * Validate the provided on-disk components (that must be for this version). @@ -152,9 +153,10 @@ public PerIndexWriter newPerIndexWriter(StorageAttachedIndex index, * This is a complete set of componentstypes that could exist on-disk. It does not imply that the * components currently exist on-disk. * + * @param hasClustering true if the SSTable forms part of a table using clustering columns * @return The set of {@link IndexComponentType} for the per-SSTable index */ - public Set perSSTableComponentTypes(); + Set perSSTableComponentTypes(boolean hasClustering); /** * Returns the set of {@link IndexComponentType} for the per-index part of an index. @@ -164,21 +166,22 @@ public PerIndexWriter newPerIndexWriter(StorageAttachedIndex index, * @param indexContext The {@link IndexContext} for the index * @return The set of {@link IndexComponentType} for the per-index index */ - default public Set perIndexComponentTypes(IndexContext indexContext) + default Set perIndexComponentTypes(IndexContext indexContext) { return perIndexComponentTypes(indexContext.getValidator()); } - public Set perIndexComponentTypes(AbstractType validator); + Set perIndexComponentTypes(AbstractType validator); /** * Return the number of open per-SSTable files that can be open during a query. * This is a static indication of the files that can be held open by an index * for queries. It is not a dynamic calculation. * + * @param hasClustering true if the SSTable forms part of a table using clustering columns * @return The number of open per-SSTable files */ - public int openFilesPerSSTable(); + int openFilesPerSSTable(boolean hasClustering); /** * Return the number of open per-index files that can be open during a query. @@ -188,7 +191,7 @@ default public Set perIndexComponentTypes(IndexContext index * @param indexContext The {@link IndexContext} for the index * @return The number of open per-index files */ - public int openFilesPerIndex(IndexContext indexContext); + int openFilesPerIndex(IndexContext indexContext); /** * Return the {@link ByteOrder} for the given {@link IndexComponentType} and {@link IndexContext}. @@ -197,7 +200,7 @@ default public Set perIndexComponentTypes(IndexContext index * @param context - The {@link IndexContext} for the index * @return The {@link ByteOrder} for the file associated with the {@link IndexComponentType} */ - public ByteOrder byteOrderFor(IndexComponentType component, IndexContext context); + ByteOrder byteOrderFor(IndexComponentType component, IndexContext context); /** * Encode the given {@link ByteBuffer} into a {@link ByteComparable} object based on the provided {@link AbstractType} @@ -217,5 +220,4 @@ default public Set perIndexComponentTypes(IndexContext index * @return the JVector file format version that this on-disk format uses. */ int jvectorFileFormatVersion(); - } diff --git a/src/java/org/apache/cassandra/index/sai/disk/format/Version.java b/src/java/org/apache/cassandra/index/sai/disk/format/Version.java index 1017cb7a18c7..0c4643823122 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/format/Version.java +++ b/src/java/org/apache/cassandra/index/sai/disk/format/Version.java @@ -146,7 +146,7 @@ private static int getAddedLengthFromDescriptorAndVersion() int addedLength = SAI_DESCRIPTOR.length() + versionNameLength + generationLength - + IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS.representation.length() + + IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS.representation.length() + SAI_SEPARATOR.length() * 3 + EXTENSION.length(); diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyFactory.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyFactory.java index 80eac7ce0cb8..020e84a86d24 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyFactory.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyFactory.java @@ -35,13 +35,6 @@ */ public class PartitionAwarePrimaryKeyFactory implements PrimaryKey.Factory { - @Override - public PrimaryKey createTokenOnly(Token token) - { - assert token != null; - return new PartitionAwarePrimaryKey(token, null, null); - } - @Override public PrimaryKey createDeferred(Token token, Supplier primaryKeySupplier) { @@ -56,7 +49,7 @@ public PrimaryKey create(DecoratedKey partitionKey, Clustering clustering) return new PartitionAwarePrimaryKey(partitionKey.getToken(), partitionKey, null); } - private class PartitionAwarePrimaryKey implements PrimaryKey + private static class PartitionAwarePrimaryKey implements PrimaryKey { private final Token token; private DecoratedKey partitionKey; @@ -72,10 +65,12 @@ private PartitionAwarePrimaryKey(Token token, DecoratedKey partitionKey, Supplie @Override public PrimaryKey loadDeferred() { - if (primaryKeySupplier != null && partitionKey == null) + if (primaryKeySupplier != null) { + assert partitionKey == null : "By definition cannot be otherwise"; this.partitionKey = primaryKeySupplier.get().partitionKey(); primaryKeySupplier = null; + assert this.token.equals(this.partitionKey.getToken()) : "Deferred primary key must contain the same token"; } return this; } @@ -156,10 +151,25 @@ public long ramBytesUsed() return shallowSize + token.getHeapSize() + preHashedDecoratedKeySize; } + /** + * Compares this primary key with another for ordering purposes. + *

+ * This implementation uses a two-tier comparison strategy: + *

    + *
  • If this partition key is null (deferred/lazy loading) or the given primary key is token only, + * compares by token only
  • + *
  • If both partition keys are available, performs full partition key comparison
  • + *
+ * Note: This comparison is partition-aware only and does not consider clustering keys. + * + * @param o the primary key to compare with + * @return a negative integer, zero, or a positive integer as this primary key is less than, + * equal to, or greater than the specified primary key + */ @Override public int compareTo(PrimaryKey o) { - if (partitionKey == null || o.partitionKey() == null) + if (partitionKey == null || o.isTokenOnly()) return token().compareTo(o.token()); return partitionKey.compareTo(o.partitionKey()); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyMap.java index d49fae12a0a8..8868284969cd 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyMap.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PartitionAwarePrimaryKeyMap.java @@ -19,16 +19,15 @@ package org.apache.cassandra.index.sai.disk.v1; import java.io.IOException; -import java.nio.ByteBuffer; import javax.annotation.concurrent.NotThreadSafe; import javax.annotation.concurrent.ThreadSafe; import org.apache.cassandra.dht.IPartitioner; import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; -import org.apache.cassandra.index.sai.disk.format.IndexComponents; -import org.apache.cassandra.index.sai.disk.format.IndexComponentType; import org.apache.cassandra.index.sai.disk.format.IndexComponent; +import org.apache.cassandra.index.sai.disk.format.IndexComponentType; +import org.apache.cassandra.index.sai.disk.format.IndexComponents; import org.apache.cassandra.index.sai.disk.v1.bitpack.BlockPackedReader; import org.apache.cassandra.index.sai.disk.v1.bitpack.MonotonicBlockPackedReader; import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; @@ -63,8 +62,8 @@ public static class PartitionAwarePrimaryKeyMapFactory implements Factory private final IndexComponents.ForRead perSSTableComponents; private final LongArray.Factory tokenReaderFactory; private final LongArray.Factory offsetReaderFactory; - private final MetadataSource metadata; private final KeyFetcher keyFetcher; + private final MetadataSource metadata; private final IPartitioner partitioner; private final PrimaryKey.Factory primaryKeyFactory; private final SSTableId sstableId; @@ -87,8 +86,8 @@ public PartitionAwarePrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComp NumericValuesMeta tokensMeta = new NumericValuesMeta(this.metadata.get(tokensComponent)); count = tokensMeta.valueCount; - token = tokensComponent.createFileHandle(); - offset = offsetsComponent.createFileHandle(); + token = tokensComponent.createFileHandle(this::close); + offset = offsetsComponent.createFileHandle(this::close); this.tokenReaderFactory = new BlockPackedReader(token, tokensMeta); this.offsetReaderFactory = new MonotonicBlockPackedReader(offset, offsetsMeta); @@ -106,8 +105,8 @@ public PartitionAwarePrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComp @Override public PrimaryKeyMap newPerSSTablePrimaryKeyMap() { - final LongArray rowIdToToken = new LongArray.DeferredLongArray(() -> tokenReaderFactory.open()); - final LongArray rowIdToOffset = new LongArray.DeferredLongArray(() -> offsetReaderFactory.open()); + final LongArray rowIdToToken = new LongArray.DeferredLongArray(tokenReaderFactory::open); + final LongArray rowIdToOffset = new LongArray.DeferredLongArray(offsetReaderFactory::open); return new PartitionAwarePrimaryKeyMap(rowIdToToken, rowIdToOffset, partitioner, keyFetcher, primaryKeyFactory, sstableId); } @@ -163,7 +162,7 @@ public PrimaryKey primaryKeyFromRowId(long sstableRowId) } @Override - public long exactRowIdOrInvertedCeiling(PrimaryKey key) + public long rowIdFromPrimaryKey(PrimaryKey key) { return rowIdToToken.indexOf(key.token().getLongValue()); } @@ -171,13 +170,7 @@ public long exactRowIdOrInvertedCeiling(PrimaryKey key) @Override public long ceiling(PrimaryKey key) { - var rowId = exactRowIdOrInvertedCeiling(key); - if (rowId >= 0) - return rowId; - if (rowId == Long.MIN_VALUE) - return -1; - else - return -rowId - 1; + return rowIdToToken.ceilingRowId(key.token().getLongValue()); } @Override diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PerIndexFiles.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PerIndexFiles.java index b82c23cc589d..2a357c370a0e 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/PerIndexFiles.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PerIndexFiles.java @@ -28,7 +28,6 @@ import org.apache.cassandra.index.sai.disk.format.IndexComponents; import org.apache.cassandra.index.sai.disk.format.IndexComponentType; -import org.apache.cassandra.index.sai.disk.format.Version; import org.apache.cassandra.io.util.FileHandle; import org.apache.cassandra.io.util.FileUtils; @@ -52,7 +51,7 @@ public PerIndexFiles(IndexComponents.ForRead perIndexComponents) { try { - files.put(component, perIndexComponents.get(component).createFileHandle()); + files.put(component, perIndexComponents.get(component).createFileHandle(this::close)); componentsPresent.add(component); } catch (UncheckedIOException e) diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java b/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java index 4901992f3fc7..1ff231b1d3b6 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/V1OnDiskFormat.java @@ -154,7 +154,7 @@ public PrimaryKey.Factory newPrimaryKeyFactory(ClusteringComparator comparator) } @Override - public PrimaryKeyMap.Factory newPrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, PrimaryKey.Factory primaryKeyFactory, SSTableReader sstable) throws IOException + public PrimaryKeyMap.Factory newPrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, PrimaryKey.Factory primaryKeyFactory, SSTableReader sstable) { return new PartitionAwarePrimaryKeyMap.PartitionAwarePrimaryKeyMapFactory(perSSTableComponents, sstable, primaryKeyFactory); } @@ -257,7 +257,7 @@ public boolean validateIndexComponent(IndexComponent.ForRead component, boolean } @Override - public Set perSSTableComponentTypes() + public Set perSSTableComponentTypes(boolean hasClustering) { return PER_SSTABLE_COMPONENTS; } @@ -271,7 +271,7 @@ public Set perIndexComponentTypes(AbstractType validator) } @Override - public int openFilesPerSSTable() + public int openFilesPerSSTable(boolean hasClustering) { return 2; } @@ -279,7 +279,8 @@ public int openFilesPerSSTable() @Override public int openFilesPerIndex(IndexContext indexContext) { - // For the V1 format there are always 2 open files per index - index (kdtree or terms) + postings + // For the V1 format there are always 2 open files per index - index (balanced tree or terms) + auxiliary postings + // for the balanced tree and postings for the literal terms return 2; } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bitpack/AbstractBlockPackedReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bitpack/AbstractBlockPackedReader.java index a8bb3b0c14a7..7f6154bf0259 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v1/bitpack/AbstractBlockPackedReader.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bitpack/AbstractBlockPackedReader.java @@ -264,5 +264,168 @@ public long length() return valueCount; } +// @Override +// public long indexOf(long targetValue) +// { +// // already out of range +// if (lastIndex >= valueCount) +// return -1; +// +// // We keep track previous returned value in lastIndex, so searching backward will not return correct result. +// // Also it's logically wrong to search backward during token iteration in PostingListRangeIterator. +// if (targetValue < previousValue) +// throw new IllegalArgumentException(String.format("%d is smaller than prev token value %d", targetValue, previousValue)); +// previousValue = targetValue; +// +// int blockIndex = binarySearchBlockMinValues(targetValue); +// +// // We need to check next block's min value on an exact match. +// boolean exactMatch = blockIndex >= 0; +// +// if (blockIndex < 0) +// { +// // A non-exact match, which is the negative index of the first value greater than the target. +// // For example, searching for 4 against min values [3,3,5,7] produces -2, which we convert to 2. +// blockIndex = -blockIndex; +// } +// +// if (blockIndex > 0) +// { +// // Start at the previous block, because there could be duplicate values in the previous block. +// // For example, with block 1: [1,2,3,3] & block 2: [3,3,5,7], binary search for 3 would find +// // block 2, but we need to start from block 1 and search both. +// // In case non-exact match, we need to pivot left as target is less than next block's min. +// blockIndex--; +// } +// +// // Find the global (not block-specific) index of the target token, which is equivalent to its row ID: +// lastIndex = findBlockRowID(targetValue, blockIndex, exactMatch); +// return lastIndex >= valueCount ? -1 : lastIndex; +// } + +// /** +// * +// * @return a positive block index for an exact match, or a negative one for a non-exact match +// */ +// private int binarySearchBlockMinValues(long targetValue) +// { +// int high = Math.toIntExact(blockBitsPerValue.length) - 1; +// +// // Assume here that we'll never move backward through the blocks: +// int low = Math.toIntExact(lastIndex >> blockShift); +// +// // Short-circuit the search if the target is in current block: +// if (low + 1 <= high) +// { +// long cmp = Long.compare(targetValue, delta(low + 1, 0)); +// +// if (cmp == 0) +// { +// // We have an exact match, so return the index of the next block, which means we'll start +// // searching from the current one and also inspect the first value of the next block. +// return low + 1; +// } +// else if (cmp < 0) +// { +// // We're in the same block. Indicate a non-exact match, and this value will be both +// // negated and then decremented to wind up at the current value of "low" here. +// return -low - 1; +// } +// +// // The target is greater than the next block's min value, so advance to that +// // block before starting the usual search... +// low++; +// } +// +// while (low <= high) +// { +// int mid = low + ((high - low) >> 1); +// +// long midVal = delta(mid, 0); +// +// if (midVal < targetValue) +// { +// low = mid + 1; +// } +// else if (midVal > targetValue) +// { +// high = mid - 1; +// } +// else +// { +// // target found, but we need to check for duplicates +// if (mid > 0 && delta(mid - 1, 0) == targetValue) +// { +// // there are duplicates, pivot left +// high = mid - 1; +// } +// else +// { +// // no duplicates +// return mid; +// } +// } +// } +// +// return -low; // no exact match found +// } + +// private long findBlockRowID(long targetValue, long blockIdx, boolean exactMatch) +// { +// // Calculate the global offset for the selected block: +// long offset = blockIdx << blockShift; +// +// // Resume from previous index if it's larger than offset +// long low = Math.max(lastIndex, offset); +// +// // The high is either the last local index in the block, or something smaller if the block isn't full: +// long high = Math.min(offset + blockMask + (exactMatch ? 1 : 0), valueCount - 1); +// +// return binarySearchBlock(targetValue, low, high); +// } + +// /** +// * binary search target value between low and high. +// * +// * @return index if exact match is found, or *positive* insertion point if no exact match is found. +// */ +// private long binarySearchBlock(long target, long low, long high) +// { +// while (low <= high) +// { +// long mid = low + ((high - low) >> 1); +// +// long midVal = get(mid); +// +// if (midVal < target) +// { +// low = mid + 1; +// // future rowId cannot be smaller than mid as long as next token not smaller than current token. +// lastIndex = mid; +// } +// else if (midVal > target) +// { +// high = mid - 1; +// } +// else +// { +// // target found, but we need to check for duplicates +// if (mid > 0 && get(mid - 1) == target) +// { +// // there are duplicates, pivot left +// high = mid - 1; +// } +// else +// { +// // exact match and no duplicates +// return mid; +// } +// } +// } +// +// // target not found +// return low; +// } + abstract long delta(int block, int idx); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyFactory.java b/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyFactory.java index 748541c476b9..41b7b23d3aea 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyFactory.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyFactory.java @@ -41,7 +41,7 @@ public class RowAwarePrimaryKeyFactory implements PrimaryKey.Factory { private final ClusteringComparator clusteringComparator; - private final boolean hasEmptyClustering; + final boolean hasEmptyClustering; public RowAwarePrimaryKeyFactory(ClusteringComparator clusteringComparator) @@ -50,12 +50,6 @@ public RowAwarePrimaryKeyFactory(ClusteringComparator clusteringComparator) this.hasEmptyClustering = clusteringComparator.size() == 0; } - @Override - public PrimaryKey createTokenOnly(Token token) - { - return new RowAwarePrimaryKey(token, null, null, null); - } - @Override public PrimaryKey createDeferred(Token token, Supplier primaryKeySupplier) { @@ -75,7 +69,7 @@ PrimaryKey createWithSource(PrimaryKeyMap primaryKeyMap, long sstableRowId, Prim private class RowAwarePrimaryKey implements PrimaryKey { - private Token token; + private final Token token; private DecoratedKey partitionKey; private Clustering clustering; private Supplier primaryKeySupplier; @@ -117,12 +111,14 @@ public Clustering clustering() @Override public PrimaryKey loadDeferred() { - if (primaryKeySupplier != null && partitionKey == null) + if (primaryKeySupplier != null ) { + assert partitionKey == null : "By definition cannot be otherwise"; PrimaryKey deferredPrimaryKey = primaryKeySupplier.get(); this.partitionKey = deferredPrimaryKey.partitionKey(); this.clustering = deferredPrimaryKey.clustering(); primaryKeySupplier = null; + assert this.token.equals(this.partitionKey.getToken()) : "Deferred primary key must contain the same token"; } return this; } @@ -153,9 +149,7 @@ private ByteSource asComparableBytes(int terminator, ByteComparable.Version vers // and clustering for the lookup loadDeferred(); - ByteSource tokenComparable = token.asComparableBytes(version); - ByteSource keyComparable = partitionKey == null ? null - : ByteSource.of(partitionKey.getKey(), version); + ByteSource keyComparable = ByteSource.of(partitionKey.getKey(), version); // It is important that the ClusteringComparator.asBytesComparable method is used // to maintain the correct clustering sort order @@ -166,14 +160,10 @@ private ByteSource asComparableBytes(int terminator, ByteComparable.Version vers .asComparableBytes(version); // prefix doesn't include null components - if (isPrefix) - { - if (keyComparable == null) - return ByteSource.withTerminator(terminator, tokenComparable); - else if (clusteringComparable == null) - return ByteSource.withTerminator(terminator, tokenComparable, keyComparable); - } - return ByteSource.withTerminator(terminator, tokenComparable, keyComparable, clusteringComparable); + if (isPrefix && clusteringComparable == null) + return ByteSource.withTerminator(terminator, keyComparable); + else + return ByteSource.withTerminator(terminator, keyComparable, clusteringComparable); } @Override @@ -182,10 +172,9 @@ public int compareTo(PrimaryKey o) int cmp = token().compareTo(o.token()); // If the tokens don't match then we don't need to compare any more of the key. - // Otherwise if this key has no deferred loader and it's partition key is null - // or the other partition key is null then one or both of the keys - // are token only so we can only compare tokens - if ((cmp != 0) || (primaryKeySupplier == null && partitionKey == null) || o.partitionKey() == null) + // Otherwise if either this key or given key are token only, + // then we can only compare tokens + if ((cmp != 0) || isTokenOnly() || o.isTokenOnly()) return cmp; // Next compare the partition keys. If they are not equal or diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyMap.java deleted file mode 100644 index 5b95076e3ea5..000000000000 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/RowAwarePrimaryKeyMap.java +++ /dev/null @@ -1,330 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.index.sai.disk.v2; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.ByteBuffer; -import javax.annotation.concurrent.NotThreadSafe; -import javax.annotation.concurrent.ThreadSafe; - -import org.apache.cassandra.db.BufferDecoratedKey; -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.ClusteringComparator; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.marshal.ByteBufferAccessor; -import org.apache.cassandra.dht.IPartitioner; -import org.apache.cassandra.dht.Token; -import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; -import org.apache.cassandra.index.sai.disk.format.IndexComponentType; -import org.apache.cassandra.index.sai.disk.format.IndexComponents; -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.apache.cassandra.index.sai.disk.v1.MetadataSource; -import org.apache.cassandra.index.sai.disk.v1.bitpack.BlockPackedReader; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsMeta; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsReader; -import org.apache.cassandra.index.sai.utils.PrimaryKey; -import org.apache.cassandra.index.sai.utils.TypeUtil; -import org.apache.cassandra.io.sstable.SSTableId; -import org.apache.cassandra.io.sstable.format.SSTableReader; -import org.apache.cassandra.io.util.FileHandle; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.Throwables; -import org.apache.cassandra.utils.bytecomparable.ByteSource; -import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; - -/** - * A row-aware {@link PrimaryKeyMap} - * - * This uses the following on-disk structures: - *
    - *
  • Block-packed structure for rowId to token lookups using {@link BlockPackedReader}. - * Uses component {@link IndexComponentType#TOKEN_VALUES}
  • - *
  • A sorted-terms structure for rowId to {@link PrimaryKey} and {@link PrimaryKey} to rowId lookups using - * {@link SortedTermsReader}. Uses components {@link IndexComponentType#PRIMARY_KEY_TRIE}, {@link IndexComponentType#PRIMARY_KEY_BLOCKS}, - * {@link IndexComponentType#PRIMARY_KEY_BLOCK_OFFSETS}
  • - *
- * - * While the {@link RowAwarePrimaryKeyMapFactory} is threadsafe, individual instances of the {@link RowAwarePrimaryKeyMap} - * are not. - */ -@NotThreadSafe -public class RowAwarePrimaryKeyMap implements PrimaryKeyMap -{ - @ThreadSafe - public static class RowAwarePrimaryKeyMapFactory implements Factory - { - private final IndexComponents.ForRead perSSTableComponents; - private final LongArray.Factory tokenReaderFactory; - private final SortedTermsReader sortedTermsReader; - private final long count; - private FileHandle token = null; - private FileHandle termsDataBlockOffsets = null; - private FileHandle termsData = null; - private FileHandle termsTrie = null; - private final IPartitioner partitioner; - private final ClusteringComparator clusteringComparator; - private final RowAwarePrimaryKeyFactory primaryKeyFactory; - private final SSTableId sstableId; - private final boolean hasStaticColumns; - - public RowAwarePrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, RowAwarePrimaryKeyFactory primaryKeyFactory, SSTableReader sstable) - { - try - { - this.perSSTableComponents = perSSTableComponents; - MetadataSource metadataSource = MetadataSource.loadMetadata(perSSTableComponents); - NumericValuesMeta tokensMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.TOKEN_VALUES))); - count = tokensMeta.valueCount; - SortedTermsMeta sortedTermsMeta = new SortedTermsMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.PRIMARY_KEY_BLOCKS))); - NumericValuesMeta blockOffsetsMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS))); - - token = perSSTableComponents.get(IndexComponentType.TOKEN_VALUES).createFileHandle(); - this.tokenReaderFactory = new BlockPackedReader(token, tokensMeta); - this.termsDataBlockOffsets = perSSTableComponents.get(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS).createFileHandle(); - this.termsData = perSSTableComponents.get(IndexComponentType.PRIMARY_KEY_BLOCKS).createFileHandle(); - this.termsTrie = perSSTableComponents.get(IndexComponentType.PRIMARY_KEY_TRIE).createFileHandle(); - this.sortedTermsReader = new SortedTermsReader(termsData, termsDataBlockOffsets, termsTrie, sortedTermsMeta, blockOffsetsMeta); - this.partitioner = sstable.metadata().partitioner; - this.primaryKeyFactory = primaryKeyFactory; - this.clusteringComparator = sstable.metadata().comparator; - this.sstableId = sstable.getId(); - this.hasStaticColumns = sstable.metadata().hasStaticColumns(); - } - catch (Throwable t) - { - throw Throwables.unchecked(Throwables.close(t, token, termsData, termsDataBlockOffsets, termsTrie)); - } - } - - @Override - public PrimaryKeyMap newPerSSTablePrimaryKeyMap() - { - final LongArray rowIdToToken = new LongArray.DeferredLongArray(() -> tokenReaderFactory.open()); - try - { - return new RowAwarePrimaryKeyMap(rowIdToToken, - sortedTermsReader, - sortedTermsReader.openCursor(), - partitioner, - primaryKeyFactory, - clusteringComparator, - sstableId, - hasStaticColumns); - } - catch (IOException e) - { - throw new UncheckedIOException(e); - } - } - - @Override - public long count() - { - return count; - } - - @Override - public void close() throws IOException - { - FileUtils.closeQuietly(token, termsData, termsDataBlockOffsets, termsTrie); - } - } - - private final LongArray rowIdToToken; - private final SortedTermsReader sortedTermsReader; - private final SortedTermsReader.Cursor cursor; - private final IPartitioner partitioner; - private final RowAwarePrimaryKeyFactory primaryKeyFactory; - private final ClusteringComparator clusteringComparator; - private final SSTableId sstableId; - private final boolean hasStaticColumns; - - private RowAwarePrimaryKeyMap(LongArray rowIdToToken, - SortedTermsReader sortedTermsReader, - SortedTermsReader.Cursor cursor, - IPartitioner partitioner, - RowAwarePrimaryKeyFactory primaryKeyFactory, - ClusteringComparator clusteringComparator, - SSTableId sstableId, - boolean hasStaticColumns) - { - this.rowIdToToken = rowIdToToken; - this.sortedTermsReader = sortedTermsReader; - this.cursor = cursor; - this.partitioner = partitioner; - this.primaryKeyFactory = primaryKeyFactory; - this.clusteringComparator = clusteringComparator; - this.sstableId = sstableId; - this.hasStaticColumns = hasStaticColumns; - } - - @Override - public SSTableId getSSTableId() - { - return sstableId; - } - - public long count() - { - return rowIdToToken.length(); - } - - @Override - public PrimaryKey primaryKeyFromRowId(long sstableRowId) - { - long token = rowIdToToken.get(sstableRowId); - return primaryKeyFactory.createDeferred(partitioner.getTokenFactory().fromLongValue(token), () -> supplier(sstableRowId)); - } - - @Override - public PrimaryKey primaryKeyFromRowId(long sstableRowId, PrimaryKey lowerBound, PrimaryKey upperBound) - { - return hasStaticColumns ? primaryKeyFromRowId(sstableRowId) - : primaryKeyFactory.createWithSource(this, sstableRowId, lowerBound, upperBound); - } - - private long skinnyExactRowIdOrInvertedCeiling(PrimaryKey key) - { - // Fast path when there is no clustering, i.e., there is one row per partition. - // (The reason we don't just make the Factory return a PartitionAware map for this case - // is that it reads partition keys directly from the sstable using the offsets file. - // While this worked in BDP, it was not efficient and caused problems because the - // sstable reader was using 64k page sizes, and this caused page cache thrashing. - long rowId = rowIdToToken.indexOf(key.token().getLongValue()); - if (rowId < 0) - // No match found, return the inverted ceiling - return rowId; - // The first index might not have been the correct match in the case of token collisions. - return tokenCollisionDetection(key, rowId); - } - - /** - * Returns a row Id for a {@link PrimaryKey}. If there is no such term, returns the `-(next row id) - 1` where - * `next row id` is the row id of the next greatest {@link PrimaryKey} in the map. - * @param key the {@link PrimaryKey} to lookup - * @return a row id - */ - @Override - public long exactRowIdOrInvertedCeiling(PrimaryKey key) - { - if (key instanceof PrimaryKeyWithSource) - { - var pkws = (PrimaryKeyWithSource) key; - if (pkws.getSourceSstableId().equals(sstableId)) - return pkws.getSourceRowId(); - } - - if (clusteringComparator.size() == 0) - return skinnyExactRowIdOrInvertedCeiling(key); - - long pointId = cursor.getExactPointId(v -> key.asComparableBytes(v)); - if (pointId >= 0) - return pointId; - long ceiling = cursor.ceiling(v -> key.asComparableBytesMinPrefix(v)); - // Use min value since -(Long.MIN_VALUE) - 1 == Long.MAX_VALUE. - return ceiling < 0 ? Long.MIN_VALUE : -ceiling - 1; - } - - @Override - public long ceiling(PrimaryKey key) - { - if (key instanceof PrimaryKeyWithSource) - { - var pkws = (PrimaryKeyWithSource) key; - if (pkws.getSourceSstableId().equals(sstableId)) - return pkws.getSourceRowId(); - } - - if (clusteringComparator.size() == 0) - { - long rowId = skinnyExactRowIdOrInvertedCeiling(key); - if (rowId >= 0) - return rowId; - else - if (rowId == Long.MIN_VALUE) - return -1; - else - return -rowId - 1; - } - - return cursor.ceiling(key::asComparableBytesMinPrefix); - } - - @Override - public long floor(PrimaryKey key) - { - return cursor.floor(key::asComparableBytesMaxPrefix); - } - - - @Override - public void close() throws IOException - { - FileUtils.closeQuietly(cursor, rowIdToToken); - } - - private PrimaryKey supplier(long sstableRowId) - { - try - { - cursor.seekToPointId(sstableRowId); - ByteSource.Peekable peekable = cursor.term().asPeekableBytes(TypeUtil.BYTE_COMPARABLE_VERSION); - - Token token = partitioner.getTokenFactory().fromComparableBytes(ByteSourceInverse.nextComponentSource(peekable), - TypeUtil.BYTE_COMPARABLE_VERSION); - byte[] keyBytes = ByteSourceInverse.getUnescapedBytes(ByteSourceInverse.nextComponentSource(peekable)); - - if (keyBytes == null) - return primaryKeyFactory.createTokenOnly(token); - - DecoratedKey partitionKey = new BufferDecoratedKey(token, ByteBuffer.wrap(keyBytes)); - - Clustering clustering = clusteringComparator.size() == 0 - ? Clustering.EMPTY - : clusteringComparator.clusteringFromByteComparable(ByteBufferAccessor.instance, - v -> ByteSourceInverse.nextComponentSource(peekable), - TypeUtil.BYTE_COMPARABLE_VERSION); - - return primaryKeyFactory.create(partitionKey, clustering); - } - catch (IOException e) - { - throw Throwables.cleaned(e); - } - } - - // Look for token collision by if the ajacent token in the token array matches the - // current token. If we find a collision we need to compare the partition key instead. - protected long tokenCollisionDetection(PrimaryKey primaryKey, long rowId) - { - // Look for collisions while we haven't reached the end of the tokens and the tokens don't collide - while (rowId + 1 < rowIdToToken.length() && primaryKey.token().getLongValue() == rowIdToToken.get(rowId + 1)) - { - // If we had a collision then see if the partition key for this row is >= to the lookup partition key - if (primaryKeyFromRowId(rowId).compareTo(primaryKey) >= 0) - return rowId; - - rowId++; - } - // Note: We would normally expect to get here without going into the while loop - return rowId; - } -} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/SSTableComponentsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v2/SSTableComponentsWriter.java index f4bfe3c34e0e..eca2a7c05362 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/SSTableComponentsWriter.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/SSTableComponentsWriter.java @@ -25,13 +25,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.db.DecoratedKey; import org.apache.cassandra.index.sai.disk.PerSSTableWriter; import org.apache.cassandra.index.sai.disk.format.IndexComponents; import org.apache.cassandra.index.sai.disk.format.IndexComponentType; import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsWriter; +import org.apache.cassandra.index.sai.disk.v2.keystore.KeyStoreWriter; import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.utils.bytecomparable.ByteSource; import org.apache.lucene.util.IOUtils; public class SSTableComponentsWriter implements PerSSTableWriter @@ -41,8 +44,12 @@ public class SSTableComponentsWriter implements PerSSTableWriter private final IndexComponents.ForWrite perSSTableComponents; private final MetadataWriter metadataWriter; private final NumericValuesWriter tokenWriter; - private final NumericValuesWriter blockFPWriter; - private final SortedTermsWriter sortedTermsWriter; + // private final NumericValuesWriter blockFPWriter; + private final NumericValuesWriter partitionSizeWriter; + private final KeyStoreWriter partitionKeysWriter; + private final KeyStoreWriter clusteringKeysWriter; + + private long partitionId = -1; public SSTableComponentsWriter(IndexComponents.ForWrite perSSTableComponents) throws IOException { @@ -51,25 +58,50 @@ public SSTableComponentsWriter(IndexComponents.ForWrite perSSTableComponents) th this.tokenWriter = new NumericValuesWriter(perSSTableComponents.addOrGet(IndexComponentType.TOKEN_VALUES), metadataWriter, false); - this.blockFPWriter = new NumericValuesWriter(perSSTableComponents.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - this.sortedTermsWriter = new SortedTermsWriter(perSSTableComponents.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - perSSTableComponents.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE)); + this.partitionSizeWriter = new NumericValuesWriter(perSSTableComponents.addOrGet(IndexComponentType.PARTITION_SIZES), metadataWriter, true); + NumericValuesWriter partitionKeyBlockOffsetWriter = new NumericValuesWriter(perSSTableComponents.addOrGet(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS), metadataWriter, true); + this.partitionKeysWriter = new KeyStoreWriter(perSSTableComponents.addOrGet(IndexComponentType.PARTITION_KEY_BLOCKS), + metadataWriter, + partitionKeyBlockOffsetWriter, + CassandraRelevantProperties.SAI_SORTED_TERMS_PARTITION_BLOCK_SHIFT.getInt(), + false); + if (perSSTableComponents.hasClustering()) + { + NumericValuesWriter clusteringKeyBlockOffsetWriter = new NumericValuesWriter(perSSTableComponents.addOrGet(IndexComponentType.CLUSTERING_KEY_BLOCK_OFFSETS), metadataWriter, true); + this.clusteringKeysWriter = new KeyStoreWriter(perSSTableComponents.addOrGet(IndexComponentType.CLUSTERING_KEY_BLOCKS), + metadataWriter, + clusteringKeyBlockOffsetWriter, + CassandraRelevantProperties.SAI_SORTED_TERMS_CLUSTERING_BLOCK_SHIFT.getInt(), + true); + } + else + { + this.clusteringKeysWriter = null; + } + } + + @Override + public void startPartition(DecoratedKey partitionKey) throws IOException + { + partitionId++; + partitionKeysWriter.add(v -> ByteSource.of(partitionKey.getKey(), v)); + if (perSSTableComponents.hasClustering()) + clusteringKeysWriter.startPartition(); } @Override public void nextRow(PrimaryKey primaryKey) throws IOException { tokenWriter.add(primaryKey.token().getLongValue()); - sortedTermsWriter.add(v -> primaryKey.asComparableBytes(v)); + partitionSizeWriter.add(partitionId); + if (perSSTableComponents.hasClustering()) + clusteringKeysWriter.add(perSSTableComponents.comparator().asByteComparable(primaryKey.clustering())); } @Override public void complete(Stopwatch stopwatch) throws IOException { - IOUtils.close(tokenWriter, sortedTermsWriter, metadataWriter); + IOUtils.close(tokenWriter, partitionSizeWriter, partitionKeysWriter, clusteringKeysWriter, metadataWriter); perSSTableComponents.markComplete(); } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/SkinnyPrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/v2/SkinnyPrimaryKeyMap.java new file mode 100644 index 000000000000..45485c2537f8 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/SkinnyPrimaryKeyMap.java @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.disk.v2; + +import org.apache.cassandra.db.BufferDecoratedKey; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.ClusteringComparator; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; +import org.apache.cassandra.index.sai.disk.format.IndexComponentType; +import org.apache.cassandra.index.sai.disk.format.IndexComponents; +import org.apache.cassandra.index.sai.disk.v1.LongArray; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.disk.v1.bitpack.BlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.bitpack.MonotonicBlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; +import org.apache.cassandra.index.sai.disk.v2.keystore.KeyLookupMeta; +import org.apache.cassandra.index.sai.disk.v2.keystore.KeyLookup; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.sstable.SSTableId; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; + +import javax.annotation.concurrent.NotThreadSafe; +import javax.annotation.concurrent.ThreadSafe; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Arrays; + +/** + * A {@link PrimaryKeyMap} for skinny tables (those with no clustering columns). + *

+ * This uses the following on-disk structures: + *

    + *
  • A block-packed structure for rowId to token lookups using {@link BlockPackedReader}. + * Uses the {@link IndexComponentType#TOKEN_VALUES} component
  • + *
  • A monotonic block packed structure for rowId to partitionId lookups using {@link MonotonicBlockPackedReader}. + * Uses the {@link IndexComponentType#PARTITION_SIZES} component
  • + *
  • A key store for rowId to {@link PrimaryKey} and {@link PrimaryKey} to rowId lookups using + * {@link KeyLookup}. Uses the {@link IndexComponentType#PARTITION_KEY_BLOCKS} and + * {@link IndexComponentType#PARTITION_KEY_BLOCK_OFFSETS} components
  • + *
+ *

+ * While the {@link Factory} is threadsafe, individual instances of the {@link SkinnyPrimaryKeyMap} + * are not. + */ +@NotThreadSafe +public class SkinnyPrimaryKeyMap implements PrimaryKeyMap +{ + @ThreadSafe + public static class Factory implements PrimaryKeyMap.Factory + { + // private final long count; + private final IndexComponents.ForRead perSSTableComponents; + private final FileHandle tokensFile; + private final ClusteringComparator clusteringComparator; + protected final SSTableId sstableId; + protected final boolean hasStaticColumns; + + protected final MetadataSource metadataSource; + protected final LongArray.Factory tokenReaderFactory; + protected final LongArray.Factory partitionReaderFactory; + protected final KeyLookup partitionKeyReader; + protected final IPartitioner partitioner; + protected final RowAwarePrimaryKeyFactory primaryKeyFactory; + + // private final FileHandle tokensFile; + private final FileHandle partitionsFile; + private final FileHandle partitionKeyBlockOffsetsFile; + private final FileHandle partitionKeyBlocksFile; + + public Factory(IndexComponents.ForRead perSSTableComponents, RowAwarePrimaryKeyFactory primaryKeyFactory, SSTableReader sstable) + { + try + { + this.perSSTableComponents = perSSTableComponents; + metadataSource = MetadataSource.loadMetadata(perSSTableComponents); + NumericValuesMeta tokensMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.TOKEN_VALUES))); + this.tokensFile = perSSTableComponents.get(IndexComponentType.TOKEN_VALUES).createFileHandle(this::close); + this.tokenReaderFactory = new BlockPackedReader(tokensFile, tokensMeta); + NumericValuesMeta partitionsMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.PARTITION_SIZES))); + this.partitionsFile = perSSTableComponents.get(IndexComponentType.PARTITION_SIZES).createFileHandle(this::close); + this.partitionReaderFactory = new MonotonicBlockPackedReader(partitionsFile, partitionsMeta); + NumericValuesMeta partitionKeyBlockOffsetsMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS))); + KeyLookupMeta partitionKeysMeta = new KeyLookupMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.PARTITION_KEY_BLOCKS))); + this.partitionKeyBlocksFile = perSSTableComponents.get(IndexComponentType.PARTITION_KEY_BLOCKS).createFileHandle(this::close); + this.partitionKeyBlockOffsetsFile = perSSTableComponents.get(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS).createFileHandle(this::close); + this.partitionKeyReader = new KeyLookup(partitionKeyBlocksFile, partitionKeyBlockOffsetsFile, partitionKeysMeta, partitionKeyBlockOffsetsMeta); + this.partitioner = sstable.metadata().partitioner; + this.primaryKeyFactory = primaryKeyFactory; + this.clusteringComparator = sstable.metadata().comparator; + this.sstableId = sstable.getId(); + this.hasStaticColumns = sstable.metadata().hasStaticColumns(); + } + catch (Throwable t) + { + throw Throwables.unchecked(t); + } + } + + @Override + @SuppressWarnings({ "resource", "RedundantSuppression" }) + public PrimaryKeyMap newPerSSTablePrimaryKeyMap() + { + LongArray rowIdToToken = new LongArray.DeferredLongArray(tokenReaderFactory::open); + LongArray rowIdToPartitionId = new LongArray.DeferredLongArray(partitionReaderFactory::open); + try + { + return new SkinnyPrimaryKeyMap(rowIdToToken, + rowIdToPartitionId, + partitionKeyReader.openCursor(), + partitioner, + primaryKeyFactory, + clusteringComparator, + sstableId, + hasStaticColumns); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + } + + @Override + public void close() + { + FileUtils.closeQuietly(Arrays.asList(tokensFile, partitionsFile, partitionKeyBlocksFile, partitionKeyBlockOffsetsFile)); + } + } + + protected final LongArray tokenArray; + protected final LongArray partitionArray; + protected final KeyLookup.Cursor partitionKeyCursor; + protected final IPartitioner partitioner; + protected final RowAwarePrimaryKeyFactory primaryKeyFactory; + protected final ClusteringComparator clusteringComparator; + private final SSTableId sstableId; + private final boolean hasStaticColumns; + + protected SkinnyPrimaryKeyMap(LongArray tokenArray, + LongArray partitionArray, + KeyLookup.Cursor partitionKeyCursor, + IPartitioner partitioner, + RowAwarePrimaryKeyFactory primaryKeyFactory, + ClusteringComparator clusteringComparator, + SSTableId sstableId, + boolean hasStaticColumns) + { + this.tokenArray = tokenArray; + this.partitionArray = partitionArray; + this.partitionKeyCursor = partitionKeyCursor; + this.partitioner = partitioner; + this.primaryKeyFactory = primaryKeyFactory; + this.clusteringComparator = clusteringComparator; + this.sstableId = sstableId; + this.hasStaticColumns = hasStaticColumns; + } + + @Override + public SSTableId getSSTableId() + { + return sstableId; + } + + @Override + public long count() + { + return tokenArray.length(); + } + + @Override + public PrimaryKey primaryKeyFromRowId(long sstableRowId) + { + long token = tokenArray.get(sstableRowId); + return primaryKeyFactory.createDeferred(partitioner.getTokenFactory().fromLongValue(token), () -> supplier(sstableRowId)); + } + + @Override + public PrimaryKey primaryKeyFromRowId(long sstableRowId, PrimaryKey lowerBound, PrimaryKey upperBound) + { + return hasStaticColumns ? primaryKeyFromRowId(sstableRowId) + : primaryKeyFactory.createWithSource(this, sstableRowId, lowerBound, upperBound); + } + + private long skinnyExactRowIdOrInvertedCeiling(PrimaryKey key) + { + // Fast path when there is no clustering, i.e., there is one row per partition. + // (The reason we don't just make the Factory return a PartitionAware map for this case + // is that it reads partition keys directly from the sstable using the offsets file. + // While this worked in BDP, it was not efficient and caused problems because the + // sstable reader was using 64k page sizes, and this caused page cache thrashing. + long rowId = tokenArray.indexOf(key.token().getLongValue()); + if (rowId < 0) + // No match found, return the inverted ceiling + return rowId; + // The first index might not have been the correct match in the case of token collisions. + return tokenCollisionDetection(key, rowId); + } + + @Override + public long rowIdFromPrimaryKey(PrimaryKey key) + { + if (key instanceof PrimaryKeyWithSource) + { + var pkws = (PrimaryKeyWithSource) key; + if (pkws.getSourceSstableId().equals(sstableId)) + return pkws.getSourceRowId(); + } + // Fast path when there is no clustering, i.e., there is one row per partition. + // (The reason we don't just make the Factory return a PartitionAware map for this case + // is that it reads partition keys directly from the sstable using the offsets file. + // While this worked in BDP, it was not efficient and caused problems because the + // sstable reader was using 64k page sizes, and this caused page cache thrashing. + long rowId = tokenArray.indexOf(key.token().getLongValue()); + // If the key is token only, the token is out of range, we are at the end of our keys, or we have skipped a token + // we can return straight away. + if (key.isTokenOnly() || rowId < 0 || rowId + 1 == tokenArray.length() || tokenArray.get(rowId) != key.token().getLongValue()) + return rowId; + // The first index might not have been the correct match in the case of token collisions. + return tokenCollisionDetection(key, rowId); + } + + // /** +// * Returns a row Id for a {@link PrimaryKey}. If there is no such term, returns the `-(next row id) - 1` where +// * `next row id` is the row id of the next greatest {@link PrimaryKey} in the map. +// * +// * @param key the {@link PrimaryKey} to lookup +// * @return a row id +// */ +// @Override +// public long exactRowIdOrInvertedCeiling(PrimaryKey key) +// { +// if (key instanceof PrimaryKeyWithSource) +// { +// var pkws = (PrimaryKeyWithSource) key; +// if (pkws.getSourceSstableId().equals(sstableId)) +// return pkws.getSourceRowId(); +// } +// +// if (clusteringComparator.size() == 0) +// return skinnyExactRowIdOrInvertedCeiling(key); +// +// long pointId = cursor.getExactPointId(v -> key.asComparableBytes(v)); +// if (pointId >= 0) +// return pointId; +// long ceiling = cursor.ceiling(v -> key.asComparableBytesMinPrefix(v)); +// // Use min value since -(Long.MIN_VALUE) - 1 == Long.MAX_VALUE. +// return ceiling < 0 ? Long.MIN_VALUE : -ceiling - 1; +// } +// @Override +// public long ceiling(PrimaryKey key) +// { +// if (key instanceof PrimaryKeyWithSource) +// { +// var pkws = (PrimaryKeyWithSource) key; +// if (pkws.getSourceSstableId().equals(sstableId)) +// return pkws.getSourceRowId(); +// } +// +// if (clusteringComparator.size() == 0) +// { +// long rowId = skinnyExactRowIdOrInvertedCeiling(key); +// if (rowId >= 0) +// return rowId; +// else if (rowId == Long.MIN_VALUE) +// return -1; +// else +// return -rowId - 1; +// } +// +// return partitionKeyCursor.ceiling(key::asComparableBytesMinPrefix); +// } + +// @Override +// public long floor(PrimaryKey key) +// { +// return cursor.floor(key::asComparableBytesMaxPrefix); +// } + + @Override + public void close() + { + FileUtils.closeQuietly(Arrays.asList(partitionKeyCursor, tokenArray, partitionArray)); + } + + // Look for token collision by if the adjacent token in the token array matches the + // current token. If we find a collision, we need to compare the partition key instead. + protected long tokenCollisionDetection(PrimaryKey primaryKey, long rowId) + { + // Look for collisions while we haven't reached the end of the tokens and the tokens don't collide + while (rowId + 1 < tokenArray.length() && primaryKey.token().getLongValue() == tokenArray.get(rowId + 1)) + { + // If we had a collision, then see if the partition key for this row is >= to the lookup partition key + if (readPartitionKey(rowId).compareTo(primaryKey.partitionKey()) >= 0) + return rowId; + + rowId++; + } + // Note: We would normally expect to get here without going into the while loop + return rowId; + } + + protected PrimaryKey supplier(long sstableRowId) + { + return primaryKeyFactory.create(readPartitionKey(sstableRowId), Clustering.EMPTY); + } + + protected DecoratedKey readPartitionKey(long sstableRowId) + { + long partitionId = partitionArray.get(sstableRowId); + ByteSource.Peekable peekable = ByteSource.peekable(partitionKeyCursor.seekToPointId(partitionId).asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + + byte[] keyBytes = ByteSourceInverse.getUnescapedBytes(peekable); + + assert keyBytes != null : "Primary key from map did not contain a partition key"; + + ByteBuffer decoratedKey = ByteBuffer.wrap(keyBytes); + return new BufferDecoratedKey(partitioner.getToken(decoratedKey), decoratedKey); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/TokenOnlyPrimaryKey.java b/src/java/org/apache/cassandra/index/sai/disk/v2/TokenOnlyPrimaryKey.java new file mode 100644 index 000000000000..02468aa299f5 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/TokenOnlyPrimaryKey.java @@ -0,0 +1,129 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.index.sai.disk.v2; + +import io.github.jbellis.jvector.util.RamUsageEstimator; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; + +public class TokenOnlyPrimaryKey implements PrimaryKey +{ + protected final Token token; + + public TokenOnlyPrimaryKey(Token token) + { + this.token = token; + } + + @Override + public boolean isTokenOnly() + { + return true; + } + + @Override + public Token token() + { + return token; + } + + @Override + public DecoratedKey partitionKey() + { + return null; + } + + @Override + public Clustering clustering() + { + return null; + } + + @Override + public ByteSource asComparableBytes(Version version) + { + return asComparableBytes(version == ByteComparable.Version.LEGACY ? ByteSource.END_OF_STREAM : ByteSource.TERMINATOR, version, false); + } + + @Override + public ByteSource asComparableBytesMinPrefix(Version version) { + return asComparableBytes(ByteSource.LT_NEXT_COMPONENT, version, true); + } + + @Override + public ByteSource asComparableBytesMaxPrefix(Version version) { + return asComparableBytes(ByteSource.GT_NEXT_COMPONENT, version, true); + } + + private ByteSource asComparableBytes(int terminator, ByteComparable.Version version, boolean isPrefix) + { + ByteSource tokenComparable = token.asComparableBytes(version); + // prefix doesn't include null components + if (isPrefix) + return ByteSource.withTerminator(terminator, tokenComparable); + else + return ByteSource.withTerminator(terminator, tokenComparable, null, null); + } + + @Override + public int compareTo(PrimaryKey o) + { + return token().compareTo(o.token()); + } + + @Override + public long ramBytesUsed() + { + // Object header + 4 references (token, partitionKey, clustering, primaryKeySupplier) + implicit outer reference + token size + return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 5L * RamUsageEstimator.NUM_BYTES_OBJECT_REF + token.getHeapSize(); + } + + @Override + public PrimaryKey forStaticRow() + { + return this; + } + + @Override + public PrimaryKey loadDeferred() + { + return this; + } + + @Override + public boolean equals(Object o) + { + if (o instanceof PrimaryKey) + return compareTo((PrimaryKey) o) == 0; + return false; + } + + @Override + public int hashCode() + { + return token().hashCode(); + } + + @Override + public String toString() + { + return String.format("PrimaryKey: { token: %s }", token()); + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/V2OnDiskFormat.java b/src/java/org/apache/cassandra/index/sai/disk/v2/V2OnDiskFormat.java index 4497f724cd84..94e834f38b2d 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/V2OnDiskFormat.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/V2OnDiskFormat.java @@ -24,6 +24,7 @@ import java.util.EnumSet; import java.util.Set; +import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,12 +52,23 @@ public class V2OnDiskFormat extends V1OnDiskFormat { private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final Set PER_SSTABLE_COMPONENTS = EnumSet.of(IndexComponentType.GROUP_COMPLETION_MARKER, - IndexComponentType.GROUP_META, - IndexComponentType.TOKEN_VALUES, - IndexComponentType.PRIMARY_KEY_TRIE, - IndexComponentType.PRIMARY_KEY_BLOCKS, - IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS); + @VisibleForTesting + public static final Set SKINNY_PER_SSTABLE_COMPONENTS = EnumSet.of(IndexComponentType.GROUP_COMPLETION_MARKER, + IndexComponentType.GROUP_META, + IndexComponentType.TOKEN_VALUES, + IndexComponentType.PARTITION_SIZES, + IndexComponentType.PARTITION_KEY_BLOCKS, + IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS); + + @VisibleForTesting + public static final Set WIDE_PER_SSTABLE_COMPONENTS = EnumSet.of(IndexComponentType.GROUP_COMPLETION_MARKER, + IndexComponentType.GROUP_META, + IndexComponentType.TOKEN_VALUES, + IndexComponentType.PARTITION_SIZES, + IndexComponentType.PARTITION_KEY_BLOCKS, + IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS, + IndexComponentType.CLUSTERING_KEY_BLOCKS, + IndexComponentType.CLUSTERING_KEY_BLOCK_OFFSETS); public static final Set VECTOR_COMPONENTS_V2 = EnumSet.of(IndexComponentType.COLUMN_COMPLETION_MARKER, IndexComponentType.META, @@ -100,7 +112,9 @@ public PrimaryKey.Factory newPrimaryKeyFactory(ClusteringComparator comparator) public PrimaryKeyMap.Factory newPrimaryKeyMapFactory(IndexComponents.ForRead perSSTableComponents, PrimaryKey.Factory primaryKeyFactory, SSTableReader sstable) { assert primaryKeyFactory instanceof RowAwarePrimaryKeyFactory; - return new RowAwarePrimaryKeyMap.RowAwarePrimaryKeyMapFactory(perSSTableComponents, (RowAwarePrimaryKeyFactory) primaryKeyFactory, sstable); + RowAwarePrimaryKeyFactory rowAwareFactory = (RowAwarePrimaryKeyFactory) primaryKeyFactory; + return rowAwareFactory.hasEmptyClustering ? new SkinnyPrimaryKeyMap.Factory(perSSTableComponents, rowAwareFactory, sstable) + : new WidePrimaryKeyMap.Factory(perSSTableComponents, rowAwareFactory, sstable); } @Override @@ -131,15 +145,20 @@ public Set perIndexComponentTypes(AbstractType validator) } @Override - public Set perSSTableComponentTypes() + public Set perSSTableComponentTypes(boolean hasClustering) { - return PER_SSTABLE_COMPONENTS; + return hasClustering ? WIDE_PER_SSTABLE_COMPONENTS : SKINNY_PER_SSTABLE_COMPONENTS; } @Override - public int openFilesPerSSTable() + public int openFilesPerSSTable(boolean hasClustering) { - return 4; + // For the V2 format the number of open files depends on whether the table has clustering. For wide tables + // the number of open files will be 6 per SSTable - token values, partition sizes index, partition key blocks, + // partition key block offsets, clustering key blocks & clustering key block offsets and for skinny tables + // the number of files will be 4 per SSTable - token values, partition key sizes, partition key blocks & + // partition key block offsets. + return hasClustering ? 6 : 4; } @Override @@ -151,7 +170,7 @@ public ByteOrder byteOrderFor(IndexComponentType indexComponentType, IndexContex case META: case GROUP_META: case TOKEN_VALUES: - case PRIMARY_KEY_BLOCK_OFFSETS: + case PARTITION_KEY_BLOCK_OFFSETS: case KD_TREE: case KD_TREE_POSTING_LISTS: return ByteOrder.LITTLE_ENDIAN; diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/V2VectorIndexSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v2/V2VectorIndexSearcher.java index 268e1b140a69..7f71ed9c41ec 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/V2VectorIndexSearcher.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/V2VectorIndexSearcher.java @@ -205,7 +205,7 @@ private CloseableIterator searchInternal(AbstractBounds + * This used the following additional on-disk structures to the {@link SkinnyPrimaryKeyMap} + *

    + *
  • A key store for rowId to {@link Clustering} and {@link Clustering} to rowId lookups using + * {@link KeyLookup}. Uses the {@link org.apache.cassandra.index.sai.disk.format.IndexComponentType#CLUSTERING_KEY_BLOCKS} and + * {@link org.apache.cassandra.index.sai.disk.format.IndexComponentType#CLUSTERING_KEY_BLOCK_OFFSETS} components
  • + *
+ * While the {@link Factory} is threadsafe, individual instances of the {@link WidePrimaryKeyMap} + * are not. + */ +@NotThreadSafe +public class WidePrimaryKeyMap extends SkinnyPrimaryKeyMap +{ + @ThreadSafe + public static class Factory extends SkinnyPrimaryKeyMap.Factory + { + private final IndexComponents.ForRead perSSTableComponents; + private final ClusteringComparator clusteringComparator; + private final KeyLookup clusteringKeyReader; + private final FileHandle clusteringKeyBlockOffsetsFile; + private final FileHandle clustingingKeyBlocksFile; + + public Factory(IndexComponents.ForRead perSSTableComponents, RowAwarePrimaryKeyFactory primaryKeyFactory, SSTableReader sstable) + { + super(perSSTableComponents, primaryKeyFactory, sstable); + + try + { + this.perSSTableComponents = perSSTableComponents; + this.clusteringKeyBlockOffsetsFile = perSSTableComponents.get(IndexComponentType.CLUSTERING_KEY_BLOCK_OFFSETS).createFileHandle(this::close); + this.clustingingKeyBlocksFile = perSSTableComponents.get(IndexComponentType.CLUSTERING_KEY_BLOCKS).createFileHandle(this::close); + this.clusteringComparator = sstable.metadata().comparator; + NumericValuesMeta clusteringKeyBlockOffsetsMeta = new NumericValuesMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.CLUSTERING_KEY_BLOCK_OFFSETS))); + KeyLookupMeta clusteringKeyMeta = new KeyLookupMeta(metadataSource.get(perSSTableComponents.get(IndexComponentType.CLUSTERING_KEY_BLOCKS))); + this.clusteringKeyReader = new KeyLookup(clustingingKeyBlocksFile, clusteringKeyBlockOffsetsFile, clusteringKeyMeta, clusteringKeyBlockOffsetsMeta); + } + catch (Throwable t) + { + throw Throwables.unchecked(t); + } + } + + @Override + @SuppressWarnings({ "resource", "RedundantSuppression" }) + public PrimaryKeyMap newPerSSTablePrimaryKeyMap() + { + LongArray rowIdToToken = new LongArray.DeferredLongArray(tokenReaderFactory::open); + LongArray partitionIdToToken = new LongArray.DeferredLongArray(partitionReaderFactory::open); + try + { + + return new WidePrimaryKeyMap(rowIdToToken, + partitionIdToToken, + partitionKeyReader.openCursor(), + clusteringKeyReader.openCursor(), + partitioner, + primaryKeyFactory, + clusteringComparator, + sstableId, + hasStaticColumns); + } + catch (IOException e) + { + throw new UncheckedIOException(e); + } + } + + @Override + public void close() + { + super.close(); + FileUtils.closeQuietly(Arrays.asList(clustingingKeyBlocksFile, clusteringKeyBlockOffsetsFile)); + } + } + + // private final ClusteringComparator clusteringComparator; + private final KeyLookup.Cursor clusteringKeyCursor; + + private WidePrimaryKeyMap(LongArray tokenArray, + LongArray partitionArray, + KeyLookup.Cursor partitionKeyCursor, + KeyLookup.Cursor clusteringKeyCursor, + IPartitioner partitioner, + RowAwarePrimaryKeyFactory primaryKeyFactory, + ClusteringComparator clusteringComparator, + SSTableId sstableId, + boolean hasStaticColumns) + { + super(tokenArray, partitionArray, partitionKeyCursor, partitioner, primaryKeyFactory, clusteringComparator, + sstableId, hasStaticColumns); + +// this.clusteringComparator = clusteringComparator; + this.clusteringKeyCursor = clusteringKeyCursor; + } + + @Override + public long rowIdFromPrimaryKey(PrimaryKey primaryKey) + { + long rowId = tokenArray.indexOf(primaryKey.token().getLongValue()); + + // If the key only has a token (initial range skip in the query), the token is out of range, + // or we have skipped a token, return the rowId from the token array. + if (primaryKey.isTokenOnly() || rowId < 0 || tokenArray.get(rowId) != primaryKey.token().getLongValue()) + return rowId; + + rowId = tokenCollisionDetection(primaryKey, rowId); + + // Search the key store for the key in the same partition + return clusteringKeyCursor.clusteredSeekToKey(clusteringComparator.asByteComparable(primaryKey.clustering()), rowId, startOfNextPartition(rowId)); + } + + @Override + public void close() + { + super.close(); + FileUtils.closeQuietly(clusteringKeyCursor); + } + + @Override + protected PrimaryKey supplier(long sstableRowId) + { + return primaryKeyFactory.create(readPartitionKey(sstableRowId), readClusteringKey(sstableRowId)); + } + + private Clustering readClusteringKey(long sstableRowId) + { + ByteSource.Peekable peekable = ByteSource.peekable(clusteringKeyCursor.seekToPointId(sstableRowId) + .asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + + Clustering clustering = clusteringComparator.clusteringFromByteComparable(ByteBufferAccessor.instance, v -> peekable); + + if (clustering == null) + clustering = Clustering.EMPTY; + + return clustering; + } + + // Returns the rowId of the next partition or the number of rows if supplied rowId is in the last partition + private long startOfNextPartition(long rowId) + { + long partitionId = partitionArray.get(rowId); + long nextPartitionRowId = partitionArray.indexOf(++partitionId); + if (nextPartitionRowId == -1) + nextPartitionRowId = partitionArray.length(); + return nextPartitionRowId; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookup.java b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookup.java new file mode 100644 index 000000000000..0056887d9587 --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookup.java @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.disk.v2.keystore; + +import java.io.IOException; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.cassandra.index.sai.disk.io.IndexInputReader; +import org.apache.cassandra.index.sai.disk.v1.LongArray; +import org.apache.cassandra.index.sai.disk.v1.bitpack.MonotonicBlockPackedReader; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; +//import org.apache.cassandra.index.sai.disk.v1.trie.TrieTermsDictionaryReader; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.Throwables; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +//import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; + +/** + * Provides read access to an on-disk sequence of partition or clustering keys written by {@link KeyStoreWriter}. + *

+ * Offers the following features: + *

    + *
  • forward iterating over all keys sequentially with a cursor
  • + *
  • constant-time look up of the key at a given point id
  • + *
  • log-time lookup of the point id of a key
  • + *
+ *

* Care has been taken to make this structure as efficient as possible. + * Reading keys does not require allocating data heap buffers per each read operation. + * Only one key at a time is loaded to memory. + * Low complexity algorithms are used – a lookup of the key by point id is constant time, + * and a lookup of the point id by the key is logarithmic. + *

+ * Because the blocks are prefix compressed, random access applies only to the locating the whole block. + * In order to jump to a concrete key inside the block, the block keys are iterated from the block beginning. + *

+ * For documentation of the underlying on-disk data structures, see the package documentation. + * + * @see KeyStoreWriter + * @see org.apache.cassandra.index.sai.disk.v2.keystore + */ +@NotThreadSafe +public class KeyLookup +{ + public static final String INDEX_OUT_OF_BOUNDS = "The target point id [%d] cannot be less than 0 or greater than or equal to the key count [%d]"; + + private final FileHandle keysFileHandle; + private final KeyLookupMeta keyLookupMeta; + private final LongArray.Factory keyBlockOffsetsFactory; + + /** + * Creates a new reader based on its data components. + *

+ * It does not own the components, so you must close them separately after you're done with the reader. + * + * @param keysFileHandle handle to the file with a sequence of prefix-compressed blocks + * each storing a fixed number of keys + * @param keysBlockOffsets handle to the file containing an encoded sequence of the file offsets pointing to the blocks + * @param keyLookupMeta metadata object created earlier by the writer + * @param keyBlockOffsetsMeta metadata object for the block offsets + */ + public KeyLookup(@Nonnull FileHandle keysFileHandle, + @Nonnull FileHandle keysBlockOffsets, + @Nonnull KeyLookupMeta keyLookupMeta, + @Nonnull NumericValuesMeta keyBlockOffsetsMeta) throws IOException + { + this.keysFileHandle = keysFileHandle; + this.keyLookupMeta = keyLookupMeta; + this.keyBlockOffsetsFactory = new MonotonicBlockPackedReader(keysBlockOffsets, keyBlockOffsetsMeta); + } + + /** + * Opens a cursor over the keys stored in the keys file. + *

+ * This will read the first key into the key buffer and point to the first point in the keys file. + *

+ * The cursor is to be used in a single thread. + * The cursor is valid as long this object hasn't been closed. + * You must close the cursor when you no longer need it. + */ + public @Nonnull Cursor openCursor() throws IOException + { + return new Cursor(keysFileHandle, keyBlockOffsetsFactory); + } + + /** + * Allows reading the keys from the keys file. + * Can quickly seek to a random key by point id. + *

+ * This object is stateful and not thread safe. + * It maintains a position to the current key as well as a buffer that can hold one key. + */ + @NotThreadSafe + public class Cursor implements AutoCloseable + { + private final IndexInputReader keysInput; + private final int blockShift; + private final int blockMask; + private final boolean clustering; + private final long keysFilePointer; + private final LongArray blockOffsets; + + // The key the cursor currently points to. Initially empty. + private final BytesRef currentKey; + + // A temporary buffer used to hold the key at the start of the next block. + private final BytesRef nextBlockKey; + + // The point id the cursor currently points to. + private long currentPointId; + private long currentBlockIndex; + + Cursor(FileHandle keysFileHandle, LongArray.Factory blockOffsetsFactory) throws IOException + { + try + { + this.keysInput = IndexInputReader.create(keysFileHandle); + SAICodecUtils.validate(this.keysInput); + this.blockShift = this.keysInput.readVInt(); + this.blockMask = (1 << this.blockShift) - 1; + this.clustering = this.keysInput.readByte() == 1; + this.keysFilePointer = this.keysInput.getFilePointer(); + this.blockOffsets = new LongArray.DeferredLongArray(blockOffsetsFactory::open); + this.currentKey = new BytesRef(keyLookupMeta.maxKeyLength); + this.nextBlockKey = new BytesRef(keyLookupMeta.maxKeyLength); + keysInput.seek(keysFilePointer); + readKey(currentPointId, currentKey); + } + catch (Throwable t) + { + if (keysFileHandle != null) + keysFileHandle.close(); + throw t; + } + } + + /** + * Finds the pointId for a clustering key within a range of pointIds. The start and end of the range must not + * exceed the number of keys available. The keys within the range are expected to be in lexographical order. + *

+ * If the key is not in the block containing the start of the range a binary search is done to find + * the block containing the search key. That block is then searched to return the pointId that corresponds + * to the key that is either equal to or next highest to the search key. + * + * @param key The key to seek for with the partition + * @param startingPointId the inclusive starting point for the partition + * @param endingPointId the exclusive ending point for the partition. + * Note: this can be equal to the number of keys if this is the last partition + * @return a {@code long} representing the pointId of the key that is >= to the key passed to the method, or + * -1 if the key passed is > all the keys. + */ + public long clusteredSeekToKey(ByteComparable key, long startingPointId, long endingPointId) + { + assert clustering : "Cannot do a clustered seek to a key on non-clustered keys"; + + BytesRef skipKey = asBytesRef(key); + + updateCurrentBlockIndex(startingPointId); + resetToCurrentBlock(); + + if (compareKeys(currentKey, skipKey) == 0) + return startingPointId; + + if (notInCurrentBlock(startingPointId, skipKey)) + { + long split = (endingPointId - startingPointId) >>> blockShift; + long splitPointId = startingPointId; + while (split > 0) + { + updateCurrentBlockIndex(Math.min((splitPointId >>> blockShift) + split, blockOffsets.length() - 1)); + resetToCurrentBlock(); + + if (currentPointId >= endingPointId) + { + updateCurrentBlockIndex((endingPointId - 1)); + resetToCurrentBlock(); + } + + int cmp = compareKeys(currentKey, skipKey); + + if (cmp == 0) + return currentPointId; + + if (cmp < 0) + splitPointId = currentPointId; + + split /= 2; + } + // After we finish the binary search we need to move the block back till we hit a block that has + // a starting key that is less than or equal to the skip key + while (currentBlockIndex > 0 && compareKeys(currentKey, skipKey) > 0) + { + currentBlockIndex--; + resetToCurrentBlock(); + } + } + + // Depending on where we are in the block we may need to move forwards to the starting point ID + while (currentPointId < startingPointId) + { + currentPointId++; + readCurrentKey(); + updateCurrentBlockIndex(currentPointId); + } + + // Move forward to the ending point ID, returning the point ID if we find our key + while (currentPointId < endingPointId) + { + if (compareKeys(currentKey, skipKey) >= 0) + return currentPointId; + currentPointId++; + if (currentPointId == keyLookupMeta.keyCount) + return -1; + readCurrentKey(); + updateCurrentBlockIndex(currentPointId); + } + return endingPointId < keyLookupMeta.keyCount ? endingPointId : -1; + } + + private void updateCurrentBlockIndex(long pointId) + { + currentBlockIndex = pointId >>> blockShift; + } + + private boolean notInCurrentBlock(long pointId, BytesRef key) + { + if (inLastBlock(pointId)) + return false; + + // Load the starting value of the next block into nextBlockKey. + long blockIndex = (pointId >>> blockShift) + 1; + long currentFp = keysInput.getFilePointer(); + keysInput.seek(blockOffsets.get(blockIndex) + keysFilePointer); + readKey(blockIndex << blockShift, nextBlockKey); + keysInput.seek(currentFp); + + return compareKeys(key, nextBlockKey) >= 0; + } + + private boolean inLastBlock(long pointId) + { + return pointId >>> blockShift == blockOffsets.length() - 1; + } + + // Reset currentPointId and currentKey to be at the start of the block + // pointed to by currentBlockIndex. + private void resetToCurrentBlock() + { + keysInput.seek(blockOffsets.get(currentBlockIndex) + keysFilePointer); + currentPointId = currentBlockIndex << blockShift; + readCurrentKey(); + } + + private void readCurrentKey() + { + readKey(currentPointId, currentKey); + } + + // Read the next key indicated by pointId. + // + // Note: pointId is only used to determine whether we are at the start of a block. It is + // important that resetPosition is called prior to multiple calls to readKey. It is + // easy to get out of position. + private void readKey(long pointId, BytesRef key) + { + try + { + int prefixLength; + int suffixLength; + if ((pointId & blockMask) == 0L) + { + prefixLength = 0; + suffixLength = keysInput.readVInt(); + } + else + { + // Read the prefix and suffix lengths following the compression mechanism described + // in the KeyStoreWriterWriter. If the lengths contained in the starting byte are less + // than the 4 bit maximum then nothing further is read. Otherwise, the lengths in the + // following vints are added. + int compressedLengths = Byte.toUnsignedInt(keysInput.readByte()); + prefixLength = compressedLengths & 0x0F; + suffixLength = compressedLengths >>> 4; + if (prefixLength == 15) + prefixLength += keysInput.readVInt(); + if (suffixLength == 15) + suffixLength += keysInput.readVInt(); + } + + assert prefixLength + suffixLength <= keyLookupMeta.maxKeyLength; + if (prefixLength + suffixLength > 0) + { + key.length = prefixLength + suffixLength; + // The currentKey is appended to as the suffix for the current key is + // added to the existing prefix. + keysInput.readBytes(key.bytes, prefixLength, suffixLength); + } + } + catch (IOException e) + { + throw Throwables.cleaned(e); + } + } + + private int compareKeys(BytesRef left, BytesRef right) + { + return FastByteOperations.compareUnsigned(left.bytes, left.offset, left.offset + left.length, + right.bytes, right.offset, right.offset + right.length); + } + + private BytesRef asBytesRef(ByteComparable source) + { + BytesRefBuilder builder = new BytesRefBuilder(); + + ByteSource byteSource = source.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION); + int val; + while ((val = byteSource.next()) != ByteSource.END_OF_STREAM) + builder.append((byte) val); + return builder.get(); + } + + /** + * Positions the cursor on the target point id and reads the key at the target to the current key buffer. + *

+ * It is allowed to position the cursor before the first item or after the last item; + * in these cases the internal buffer is cleared. + * + * @param target point id to lookup + * @return The {@link ByteComparable} containing the key + * @throws IndexOutOfBoundsException if the target point id is less than -1 or greater than the number of keys + */ + public @Nonnull ByteComparable seekToPointId(long target) + { +// if (target < -1 || target > keyLookupMeta.keyCount) + if (target <= -1 || target >= keyLookupMeta.keyCount) + throw new IndexOutOfBoundsException(String.format(INDEX_OUT_OF_BOUNDS, target, keyLookupMeta.keyCount)); + +// if (target == -1 || target == keyLookupMeta.keyCount) + + if (target != currentPointId) + { + long blockIndex = target >>> blockShift; + // We need to reset the block if the block index has changed or the target < currentPointId. + // We can read forward in the same block without a reset, but we can't read backwards, and token + // collision can result in us moving backwards. + if (blockIndex != currentBlockIndex || target < currentPointId) + { + currentBlockIndex = blockIndex; + resetToCurrentBlock(); + } + } + while (currentPointId < target) + { + currentPointId++; + readCurrentKey(); + updateCurrentBlockIndex(currentPointId); + } + + return ByteComparable.preencoded(TypeUtil.BYTE_COMPARABLE_VERSION, currentKey.bytes, currentKey.offset, currentKey.length); + } + + @VisibleForTesting + public void reset() throws IOException + { + currentPointId = 0; + currentBlockIndex = 0; + keysInput.seek(keysFilePointer); + readCurrentKey(); + } + + @Override + public void close() throws IOException + { + blockOffsets.close(); + keysInput.close(); + } + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsMeta.java b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupMeta.java similarity index 51% rename from src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsMeta.java rename to src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupMeta.java index 4350eb564209..10992d4f04e6 100644 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsMeta.java +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupMeta.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.cassandra.index.sai.disk.v2.sortedterms; +package org.apache.cassandra.index.sai.disk.v2.keystore; import java.io.IOException; @@ -24,33 +24,33 @@ import org.apache.lucene.store.IndexOutput; /** - * Metadata produced by {@link SortedTermsWriter}, needed by {@link SortedTermsReader}. + * Metadata produced by {@link KeyStoreWriter}, needed by {@link KeyLookup}. */ -public class SortedTermsMeta +public class KeyLookupMeta { - public final long trieFP; - /** Number of terms */ - public final long count; - public final int maxTermLength; +// public final long trieFP; + public final long keyCount; + public final int maxKeyLength; - public SortedTermsMeta(IndexInput input) throws IOException + public KeyLookupMeta(IndexInput input) throws IOException { - this.trieFP = input.readLong(); - this.count = input.readLong(); - this.maxTermLength = input.readInt(); +// this.trieFP = input.readLong(); + this.keyCount = input.readLong(); + this.maxKeyLength = input.readInt(); } - public SortedTermsMeta(long trieFP, long count, int maxTermLength) + public KeyLookupMeta(long keyCount, int maxKeyLength) +// public KeyLookupMeta(long trieFP, long keyCount, int maxKeyLength) { - this.trieFP = trieFP; - this.count = count; - this.maxTermLength = maxTermLength; +// this.trieFP = trieFP; + this.keyCount = keyCount; + this.maxKeyLength = maxKeyLength; } - public void write(IndexOutput output) throws IOException + public static void write(IndexOutput output, long keyCount, int maxKeyLength) throws IOException { - output.writeLong(trieFP); - output.writeLong(count); - output.writeInt(maxTermLength); +// output.writeLong(trieFP); + output.writeLong(keyCount); + output.writeInt(maxKeyLength); } } diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyStoreWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyStoreWriter.java new file mode 100644 index 000000000000..f4f8daec11ca --- /dev/null +++ b/src/java/org/apache/cassandra/index/sai/disk/v2/keystore/KeyStoreWriter.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.disk.v2.keystore; + +import java.io.Closeable; +import java.io.IOException; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.NotThreadSafe; + +import io.micrometer.core.lang.NonNull; +import org.apache.cassandra.index.sai.disk.format.IndexComponent; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.utils.FastByteOperations; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.apache.lucene.util.StringHelper; + +/** + * Writes a sequence of partition keys or clustering keys for use with {@link KeyLookup}. + *

+ * Partition keys are written unordered and clustering keys are written in ordered partitions determined by calls to + * {@link #startPartition()}. In either case keys can be of varying lengths. + *

+ * The {@link #blockShift} field is used to quickly determine the id of the current block + * based on a point id or to check if we are exactly at the beginning of the block. + *

+ * Keys are organized in blocks of (2 ^ {@link #blockShift}) keys. + *

+ * The blocks should not be too small because they allow prefix compression of the keys except the first key in a block. + *

+ * The blocks should not be too large because we can't just randomly jump to the key inside the block, but we have to + * iterate through all the keys from the start of the block. + * + * @see KeyLookup + */ +@NotThreadSafe +public class KeyStoreWriter implements Closeable +{ + private final int blockShift; + private final int blockMask; + private final boolean clustering; + private final IndexOutput keysOutput; + private final NumericValuesWriter offsetsWriter; + private final String componentName; + private final MetadataWriter metadataWriter; + + private BytesRefBuilder prevKey = new BytesRefBuilder(); + private BytesRefBuilder tempKey = new BytesRefBuilder(); + + private final long bytesStartFP; + + private boolean inPartition = false; + private int maxKeyLength = -1; + private long pointId = 0; + + /** + * Creates a new writer. + *

+ * It does not own the components, so you must close the components by yourself + * after you're done with the writer. + * + * @param keysDataComponent component builder for the prefix-compressed keys data + * @param metadataWriter the {@link MetadataWriter} for storing the {@link KeyLookupMeta} + * @param keysBlockOffsets where to write the offsets of each block of keys + * @param blockShift the block shift that is used to determine the block size + * @param clustering determines whether the keys will be written as ordered partitions + */ + public KeyStoreWriter(@NonNull IndexComponent.ForWrite keysDataComponent, + @NonNull MetadataWriter metadataWriter, +// @NonNull IndexOutput keysOutput, + @NonNull NumericValuesWriter keysBlockOffsets, + int blockShift, + boolean clustering) throws IOException + { + this.componentName = keysDataComponent.fileNamePart(); + this.metadataWriter = metadataWriter; + this.blockShift = blockShift; + this.blockMask = (1 << this.blockShift) - 1; + this.clustering = clustering; + this.keysOutput = keysDataComponent.openOutput(); + SAICodecUtils.writeHeader(this.keysOutput); + this.keysOutput.writeVInt(blockShift); + this.keysOutput.writeByte((byte) (clustering ? 1 : 0)); + this.bytesStartFP = keysOutput.getFilePointer(); + this.offsetsWriter = keysBlockOffsets; + } + + public void startPartition() + { + assert clustering : "Cannot start a partition on a non-clustering key store"; + + inPartition = false; + } + + /** + * Appends a key at the end of the sequence. + * + * @throws IOException if write to disk fails + * @throws IllegalArgumentException if the key is not greater than the previous added key + */ + public void add(final @Nonnull ByteComparable key) throws IOException + { + tempKey.clear(); + copyBytes(key, tempKey); + + BytesRef keyRef = tempKey.get(); + + if (clustering && inPartition) + { + if (compareKeys(keyRef, prevKey.get()) <= 0) + throw new IllegalArgumentException("Clustering keys must be in ascending lexographical order"); + } + + inPartition = true; + + writeKey(keyRef); + + swapTempWithPrevious(); + maxKeyLength = Math.max(maxKeyLength, keyRef.length); + pointId++; + } + + private void writeKey(BytesRef key) throws IOException + { + if ((pointId & blockMask) == 0) + { + offsetsWriter.add(keysOutput.getFilePointer() - bytesStartFP); + + keysOutput.writeVInt(key.length); + keysOutput.writeBytes(key.bytes, key.offset, key.length); + } + else + { + int prefixLength = 0; + int suffixLength = 0; + + // If the key is the same as the previous key then we use prefix and suffix lengths of 0. + // This means that we store a byte of 0 and don't write any data for the key. + if (compareKeys(prevKey.get(), key) != 0) + { + prefixLength = StringHelper.bytesDifference(prevKey.get(), key); + suffixLength = key.length - prefixLength; + } + // The prefix and suffix lengths are written as a byte followed by up to 2 vints. An attempt is + // made to compress the lengths into the byte (if prefix length < 15 and/or suffix length < 15). + // If either length exceeds the compressed byte maximum, it is written as a vint following the byte. + keysOutput.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength) << 4))); + + if (prefixLength + suffixLength > 0) + { + if (prefixLength >= 15) + keysOutput.writeVInt(prefixLength - 15); + if (suffixLength >= 15) + keysOutput.writeVInt(suffixLength - 15); + + keysOutput.writeBytes(key.bytes, key.offset + prefixLength, key.length - prefixLength); + } + } + } + + /** + * Flushes any in-memory buffers to the output streams. + * Does not close the output streams. + * No more writes are allowed. + */ + @Override + public void close() throws IOException + { + try (IndexOutput output = metadataWriter.builder(componentName)) + { + SAICodecUtils.writeFooter(keysOutput); + KeyLookupMeta.write(output, pointId, maxKeyLength); + } + finally + { + FileUtils.close(offsetsWriter, keysOutput); + } + } + + private int compareKeys(BytesRef left, BytesRef right) + { + return FastByteOperations.compareUnsigned(left.bytes, left.offset, left.offset + left.length, + right.bytes, right.offset, right.offset + right.length); + } + + /** + * Copies bytes from source to dest. + */ + private void copyBytes(ByteComparable source, BytesRefBuilder dest) + { + ByteSource byteSource = source.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION); + int val; + while ((val = byteSource.next()) != ByteSource.END_OF_STREAM) + dest.append((byte) val); + } + + /** + * Swaps this.temp with this.previous. + * It is faster to swap the pointers instead of copying the data. + */ + private void swapTempWithPrevious() + { + BytesRefBuilder temp = this.tempKey; + this.tempKey = this.prevKey; + this.prevKey = temp; + } +} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsReader.java b/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsReader.java deleted file mode 100644 index c7465062edfa..000000000000 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsReader.java +++ /dev/null @@ -1,331 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.index.sai.disk.v2.sortedterms; - -import java.io.IOException; -import javax.annotation.Nonnull; -import javax.annotation.concurrent.NotThreadSafe; -import javax.annotation.concurrent.ThreadSafe; - -import com.google.common.base.Preconditions; - -import org.apache.cassandra.index.sai.disk.io.IndexInputReader; -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.apache.cassandra.index.sai.disk.v1.bitpack.MonotonicBlockPackedReader; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; -import org.apache.cassandra.index.sai.disk.v1.trie.TrieTermsDictionaryReader; -import org.apache.cassandra.index.sai.utils.SAICodecUtils; -import org.apache.cassandra.index.sai.utils.TypeUtil; -import org.apache.cassandra.io.util.FileHandle; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.util.BytesRef; - -import static org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsWriter.TERMS_DICT_BLOCK_MASK; -import static org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsWriter.TERMS_DICT_BLOCK_SHIFT; - -/** - * Provides read access to a sorted on-disk sequence of terms. - *

- * Offers the following features: - *

    - *
  • forward iterating over all terms sequentially with a cursor
  • - *
  • constant-time look up of the term at a given point id
  • - *
  • log-time lookup of the point id of a term
  • - *
- *

- * Care has been taken to make this structure as efficient as possible. - * Reading terms does not require allocating data heap buffers per each read operation. - * Only one term at a time is loaded to memory. - * Low complexity algorithms are used – a lookup of the term by point id is constant time, - * and a lookup of the point id by the term is logarithmic. - * - *

- * Because the blocks are prefix compressed, random access applies only to the locating the whole block. - * In order to jump to a concrete term inside the block, the block terms are iterated from the block beginning. - * Expect random access by {@link Cursor#seekToPointId(long)} to be slower - * than just moving to the next term with {@link Cursor#advance()}. - *

- * For documentation of the underlying on-disk data structures, see the package documentation. - * - * @see SortedTermsWriter - * @see org.apache.cassandra.index.sai.disk.v2.sortedterms - */ -@ThreadSafe -public class SortedTermsReader -{ - private final FileHandle termsData; - private final SortedTermsMeta meta; - private final FileHandle termsTrie; - private final LongArray.Factory blockOffsetsFactory; - - /** - * Creates a new reader based on its data components. - *

- * It does not own the components, so you must close them separately after you're done with the reader. - * @param termsData handle to the file with a sequence of prefix-compressed blocks - * each storing a fixed number of terms - * @param termsDataBlockOffsets handle to the file containing an encoded sequence of the file offsets pointing to the blocks - * @param termsTrie handle to the file storing the trie with the term-to-point-id mapping - * @param meta metadata object created earlier by the writer - * @param blockOffsetsMeta metadata object for the block offsets - */ - public SortedTermsReader(@Nonnull FileHandle termsData, - @Nonnull FileHandle termsDataBlockOffsets, - @Nonnull FileHandle termsTrie, - @Nonnull SortedTermsMeta meta, - @Nonnull NumericValuesMeta blockOffsetsMeta) throws IOException - { - this.termsData = termsData; - this.termsTrie = termsTrie; - try (IndexInput trieInput = IndexInputReader.create(termsTrie)) - { - SAICodecUtils.validate(trieInput); - } - this.meta = meta; - this.blockOffsetsFactory = new MonotonicBlockPackedReader(termsDataBlockOffsets, blockOffsetsMeta); - } - - /** - * Returns the total number of terms. - */ - public long count() - { - return meta.count; - } - - /** - * Opens a cursor over the terms stored in the terms file. - *

- * This does not read any data yet. - * The cursor is initially positioned before the first item. - *

- * The cursor is to be used in a single thread. - * The cursor is valid as long this object hasn't been closed. - * You must close the cursor when you no longer need it. - */ - public @Nonnull Cursor openCursor() throws IOException - { - return new Cursor(termsData, blockOffsetsFactory); - } - - /** - * Allows reading the terms from the terms file. - * Can quickly seek to a random term by pointId. - *

- * This object is stateful and not thread safe. - * It maintains a position to the current term as well as a buffer that can hold one term. - */ - @NotThreadSafe - public class Cursor implements AutoCloseable - { - private final IndexInputReader termsData; - private final long termsDataFp; - private final LongArray blockOffsets; - - // The term the cursor currently points to. Initially empty. - private final BytesRef currentTerm; - - // The point id the cursor currently points to. -1 means before the first item. - private long pointId = -1; - - private TrieTermsDictionaryReader reader; - - Cursor(FileHandle termsData, LongArray.Factory blockOffsetsFactory) throws IOException - { - try - { - this.termsData = IndexInputReader.create(termsData); - SAICodecUtils.validate(this.termsData); - this.termsDataFp = this.termsData.getFilePointer(); - this.blockOffsets = new LongArray.DeferredLongArray(blockOffsetsFactory::open); - this.currentTerm = new BytesRef(Math.max(meta.maxTermLength, 0)); // maxTermLength can be negative if meta.count == 0 - this.reader = new TrieTermsDictionaryReader(termsTrie.instantiateRebufferer(), meta.trieFP, TypeUtil.BYTE_COMPARABLE_VERSION); - } - catch (Throwable t) - { - if (termsData != null) - termsData.close(); - throw t; - } - } - - /** - * Returns the point id (ordinal) associated with the least term greater than or equal to the given term, or - * a negative value if there is no such term. - * @param term - * @return - */ - public long ceiling(@Nonnull ByteComparable term) - { - Preconditions.checkNotNull(term, "term null"); - return reader.ceiling(term); - } - - /** - * Returns the point id (ordinal) of the target term or a negative value if there is no such term. - * Complexity of this operation is O(log n). - * - * @param term target term to lookup - */ - public long getExactPointId(@Nonnull ByteComparable term) - { - Preconditions.checkNotNull(term, "term null"); - return reader.exactMatch(term); - } - - /** - * Returns the point id (ordinal) associated with the greatest term less than or equal to the given term, or - * a negative value if there is no such term. - * Complexity of this operation is O(log n). - * - * @param term target term to lookup - */ - public long floor(@Nonnull ByteComparable term) - { - Preconditions.checkNotNull(term, "term null"); - return reader.floor(term); - } - - /** - * Returns the number of terms - */ - public long count() - { - return SortedTermsReader.this.count(); - } - - /** - * Returns the current position of the cursor. - * Initially, before the first call to {@link Cursor#advance}, the cursor is positioned at -1. - * After reading all the items, the cursor is positioned at index one - * greater than the position of the last item. - */ - public long pointId() - { - return pointId; - } - - /** - * Returns the current term data as ByteComparable referencing the internal term buffer. - * The term data stored behind that reference is valid only until the next call to - * {@link Cursor#advance} or {@link Cursor#seekToPointId(long)}. - */ - public @Nonnull ByteComparable term() - { - return ByteComparable.preencoded(reader.byteComparableVersion, currentTerm.bytes, currentTerm.offset, currentTerm.length); - } - - /** - * Advances the cursor to the next term and reads it into the current term buffer. - *

- * If there are no more available terms, clears the term buffer and the cursor's position will point to the - * one behind the last item. - *

- * This method has constant time complexity. - * - * @return true if the cursor was advanced successfully, false if the end of file was reached - * @throws IOException if a read from the terms file fails - */ - public boolean advance() throws IOException - { - if (pointId >= meta.count || ++pointId >= meta.count) - { - currentTerm.length = 0; - return false; - } - - int prefixLength; - int suffixLength; - if ((pointId & TERMS_DICT_BLOCK_MASK) == 0L) - { - prefixLength = 0; - suffixLength = termsData.readVInt(); - } - else - { - final int token = Byte.toUnsignedInt(termsData.readByte()); - prefixLength = token & 0x0F; - suffixLength = 1 + (token >>> 4); - if (prefixLength == 15) - prefixLength += termsData.readVInt(); - if (suffixLength == 16) - suffixLength += termsData.readVInt(); - } - - assert prefixLength + suffixLength <= meta.maxTermLength; - currentTerm.length = prefixLength + suffixLength; - termsData.readBytes(currentTerm.bytes, prefixLength, suffixLength); - return true; - } - - /** - * Positions the cursor on the target point id and reads the term at target to the current term buffer. - *

- * It is allowed to position the cursor before the first item or after the last item; - * in these cases the internal buffer is cleared. - *

- * This method has constant complexity. - * - * @param target point id to lookup - * @throws IOException if a seek and read from the terms file fails - * @throws IndexOutOfBoundsException if the target point id is less than -1 or greater than {@link Cursor#count}. - */ - public void seekToPointId(long target) throws IOException - { - if (target < -1 || target > meta.count) - throw new IndexOutOfBoundsException(); - - if (target == -1 || target == meta.count) - { - termsData.seek(termsDataFp); // matters only if target is -1 - pointId = target; - currentTerm.length = 0; - } - else - { - final long blockIndex = target >>> TERMS_DICT_BLOCK_SHIFT; - final long blockAddress = blockOffsets.get(blockIndex); - termsData.seek(blockAddress + termsDataFp); - pointId = (blockIndex << TERMS_DICT_BLOCK_SHIFT) - 1; - while (pointId < target) - { - boolean advanced = advance(); - assert advanced : "unexpected eof"; // must return true because target is in range - } - } - } - - /** - * Resets the cursor to its initial position before the first item. - */ - public void reset() throws IOException - { - seekToPointId(-1); - } - - @Override - public void close() throws IOException - { - blockOffsets.close(); - termsData.close(); - reader.close(); - } - } -} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsWriter.java deleted file mode 100644 index 6c18f7979040..000000000000 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsWriter.java +++ /dev/null @@ -1,218 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.index.sai.disk.v2.sortedterms; - -import java.io.Closeable; -import java.io.IOException; -import javax.annotation.Nonnull; -import javax.annotation.concurrent.NotThreadSafe; - -import com.google.common.base.Preconditions; - -import io.micrometer.core.lang.NonNull; -import org.apache.cassandra.index.sai.disk.format.IndexComponent; -import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter; -import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; -import org.apache.cassandra.index.sai.utils.SAICodecUtils; -import org.apache.cassandra.index.sai.utils.TypeUtil; -import org.apache.cassandra.io.tries.IncrementalTrieWriter; -import org.apache.cassandra.io.util.FileUtils; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.BytesRefBuilder; -import org.apache.lucene.util.StringHelper; - -import static org.apache.cassandra.index.sai.disk.v1.trie.TrieTermsDictionaryReader.trieSerializer; - -/** - * Writes an ordered sequence of terms for use with {@link SortedTermsReader}. - *

- * Terms must be added in lexicographical ascending order. - * Terms can be of varying lengths. - * - *

- * Important implementation note: SAI blocked packed readers are slow, - * and Lucene MonotonicBlockPackedReader is slow. Using them - * will cause this class to slow considerably. - * - * For documentation of the underlying on-disk data structures, see the package documentation. - * - * @see SortedTermsReader - * @see org.apache.cassandra.index.sai.disk.v2.sortedterms - */ -@NotThreadSafe -public class SortedTermsWriter implements Closeable -{ - // The TERMS_DICT_ constants allow for quickly determining the id of the current block based on a point id - // or to check if we are exactly at the beginning of the block. - // Terms data are organized in blocks of (2 ^ TERMS_DICT_BLOCK_SHIFT) terms. - // The blocks should not be too small because they allow prefix compression of - // the terms except the first term in a block. - // The blocks should not be too large because we can't just ranfomly jump to the term inside the block, - // but we have to iterate through all the terms from the start of the block. - static final int TERMS_DICT_BLOCK_SHIFT = 4; - static final int TERMS_DICT_BLOCK_SIZE = 1 << TERMS_DICT_BLOCK_SHIFT; - static final int TERMS_DICT_BLOCK_MASK = TERMS_DICT_BLOCK_SIZE - 1; - - static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16; - - private final IncrementalTrieWriter trieWriter; - private final IndexOutputWriter trieOutput; - private final IndexOutput termsOutput; - private final NumericValuesWriter offsetsWriter; - private final String componentName; - private final MetadataWriter metadataWriter; - - private BytesRefBuilder prevTerm = new BytesRefBuilder(); - private BytesRefBuilder tempTerm = new BytesRefBuilder(); - - private final long bytesStartFP; - - private int maxLength = -1; - private long pointId = 0; - - /** - * Creates a new writer. - *

- * It does not own the components, so you must close the components by yourself - * after you're done with the writer. - * - * @param termsDataComponent component builder for the prefix-compressed terms data - * @param metadataWriter the MetadataWriter for storing the SortedTermsMeta - * @param termsDataBlockOffsets where to write the offsets of each block of terms data - * @param trieComponent component where to write the trie that maps the terms to point ids - */ - public SortedTermsWriter(@NonNull IndexComponent.ForWrite termsDataComponent, - @NonNull MetadataWriter metadataWriter, - @Nonnull NumericValuesWriter termsDataBlockOffsets, - @Nonnull IndexComponent.ForWrite trieComponent) throws IOException - { - this.componentName = termsDataComponent.fileNamePart(); - this.metadataWriter = metadataWriter; - this.trieOutput = trieComponent.openOutput(); - SAICodecUtils.writeHeader(this.trieOutput); - this.trieWriter = IncrementalTrieWriter.open(trieSerializer, trieOutput.asSequentialWriter(), TypeUtil.BYTE_COMPARABLE_VERSION); - this.termsOutput = termsDataComponent.openOutput(); - SAICodecUtils.writeHeader(termsOutput); - this.bytesStartFP = termsOutput.getFilePointer(); - this.offsetsWriter = termsDataBlockOffsets; - } - - /** - * Appends a term at the end of the sequence. - * Terms must be added in lexicographic order. - * - * @throws IOException if write to disk fails - * @throws IllegalArgumentException if the term is not greater than the previous added term - */ - public void add(final @Nonnull ByteComparable term) throws IOException - { - tempTerm.clear(); - copyBytes(term, tempTerm); - - final BytesRef termRef = tempTerm.get(); - final BytesRef prevTermRef = this.prevTerm.get(); - - Preconditions.checkArgument(prevTermRef.length == 0 || prevTermRef.compareTo(termRef) < 0, - "Terms must be added in lexicographic ascending order."); - writeTermData(termRef); - writeTermToTrie(term); - - maxLength = Math.max(maxLength, termRef.length); - swapTempWithPrevious(); - pointId++; - } - - private void writeTermToTrie(ByteComparable term) throws IOException - { - trieWriter.add(term, pointId); - } - - private void writeTermData(BytesRef term) throws IOException - { - if ((pointId & TERMS_DICT_BLOCK_MASK) == 0) - { - offsetsWriter.add(termsOutput.getFilePointer() - bytesStartFP); - - termsOutput.writeVInt(term.length); - termsOutput.writeBytes(term.bytes, term.offset, term.length); - } - else - { - final int prefixLength = StringHelper.bytesDifference(prevTerm.get(), term); - final int suffixLength = term.length - prefixLength; - assert suffixLength > 0: "terms must be unique"; - - termsOutput.writeByte((byte) (Math.min(prefixLength, 15) | (Math.min(15, suffixLength - 1) << 4))); - if (prefixLength >= 15) - termsOutput.writeVInt(prefixLength - 15); - if (suffixLength >= 16) - termsOutput.writeVInt(suffixLength - 16); - - termsOutput.writeBytes(term.bytes, term.offset + prefixLength, term.length - prefixLength); - } - } - - /** - * Flushes any in-memory buffers to the output streams. - * Does not close the output streams. - * No more writes are allowed. - */ - @Override - public void close() throws IOException - { - try (IndexOutput output = metadataWriter.builder(componentName)) - { - final long trieFP = this.trieWriter.complete(); - SAICodecUtils.writeFooter(trieOutput); - SAICodecUtils.writeFooter(termsOutput); - SortedTermsMeta sortedTermsMeta = new SortedTermsMeta(trieFP, pointId, maxLength); - sortedTermsMeta.write(output); - } - finally - { - FileUtils.closeQuietly(trieWriter, trieOutput, termsOutput, offsetsWriter); - } - } - - /** - * Copies bytes from source to dest. - */ - private void copyBytes(ByteComparable source, BytesRefBuilder dest) - { - ByteSource byteSource = source.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION); - int val; - while ((val = byteSource.next()) != ByteSource.END_OF_STREAM) - dest.append((byte) val); - } - - /** - * Swaps this.temp with this.previous. - * It is faster to swap the pointers instead of copying the data. - */ - private void swapTempWithPrevious() - { - BytesRefBuilder temp = this.tempTerm; - this.tempTerm = this.prevTerm; - this.prevTerm = temp; - } -} diff --git a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/package-info.java b/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/package-info.java deleted file mode 100644 index d5f1cf9f3eef..000000000000 --- a/src/java/org/apache/cassandra/index/sai/disk/v2/sortedterms/package-info.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -/** - * Space-efficient on-disk data structure for storing a sorted sequence of terms. - * Provides efficient lookup of terms by their point id, as well as locating them by contents. - *

- * All the code in the package uses the following teminology: - *

    - *
  • Term: arbitrary data provided by the user as a bunch of bytes. Terms can be of variable length.
  • - *
  • Point id: the ordinal position of a term in the sequence, 0-based.
  • - *
- * - * Terms are stored in ByteComparable strictly ascending order. - * Duplicates are not allowed. - * - *

- * The structure is immutable, i.e. cannot be modified nor appended after writing to disk is completed. - * You build it by adding terms in the ascending order using - * {@link org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsWriter}. - * Once saved to disk, you can open it for lookups with - * {@link org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsReader}. - * - *

- * The data structure comprises of the following components, each stored in a separate file: - *

    - *
  • terms data, organized as a sequence of prefix-compressed blocks each storing 16 terms
  • - *
  • a monotonic list of file offsets of the blocks; this component allows to quickly locate the block - * that contains the term with a given point id
  • - *
  • a trie indexed by terms, with a long payload for the point id, - * to quickly locate the point id of a term by the term contents - *
  • - *
- *

- * - * The implementation has been based on code from Lucene version 7.5 SortedDocValues. - * Prefix compression and bitpacking are used extensively to save space. - */ -package org.apache.cassandra.index.sai.disk.v2.sortedterms; \ No newline at end of file diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java index 800ca7bccdb4..8594107b72e5 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIterator.java @@ -40,6 +40,7 @@ public class KeyRangeConcatIterator extends KeyRangeIterator { private final Iterator ranges; private KeyRangeIterator currentRange; + public static final String MUST_BE_SORTED_ERROR = "RangeIterator must be sorted, previous max: %s, next min: %s"; private final List toRelease; protected KeyRangeConcatIterator(KeyRangeIterator.Builder.Statistics statistics, List ranges) diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java index 340a1f5ab9be..2b0199dae143 100644 --- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java +++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeUnionIterator.java @@ -125,7 +125,7 @@ private void maybeSkipCurrentPartition() private void skipPartition(KeyRangeIterator iterator, DecoratedKey partitionKey) { // TODO: Push this logic down to the iterator where it can be more efficient - while (iterator.hasNext() && iterator.peek().partitionKey() != null && iterator.peek().partitionKey().compareTo(partitionKey) <= 0) + while (iterator.hasNext() && !iterator.peek().isTokenOnly() && iterator.peek().partitionKey().compareTo(partitionKey) <= 0) iterator.next(); } diff --git a/src/java/org/apache/cassandra/index/sai/memory/MemtableKeyRangeIterator.java b/src/java/org/apache/cassandra/index/sai/memory/MemtableKeyRangeIterator.java index 854641022b16..bc8fab194c8a 100644 --- a/src/java/org/apache/cassandra/index/sai/memory/MemtableKeyRangeIterator.java +++ b/src/java/org/apache/cassandra/index/sai/memory/MemtableKeyRangeIterator.java @@ -90,9 +90,9 @@ private static PrimaryKey maxKey(Memtable memtable, PrimaryKey.Factory factory) @Override protected void performSkipTo(PrimaryKey nextKey) { - PartitionPosition start = nextKey.partitionKey() != null - ? nextKey.partitionKey() - : nextKey.token().minKeyBound(); + PartitionPosition start = nextKey.isTokenOnly() + ? nextKey.token().minKeyBound() + : nextKey.partitionKey(); if (!keyRange.right.isMinimum() && start.compareTo(keyRange.right) > 0) { partitionIterator = EmptyIterators.unfilteredPartition(memtable.metadata()); diff --git a/src/java/org/apache/cassandra/index/sai/utils/IndexFileUtils.java b/src/java/org/apache/cassandra/index/sai/utils/IndexFileUtils.java index 86e0ca7b4cba..97d9c04c2f98 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/IndexFileUtils.java +++ b/src/java/org/apache/cassandra/index/sai/utils/IndexFileUtils.java @@ -51,7 +51,7 @@ public class IndexFileUtils protected static final Logger logger = LoggerFactory.getLogger(IndexFileUtils.class); @VisibleForTesting - protected static final SequentialWriterOption defaultWriterOption = SequentialWriterOption.newBuilder() + public static final SequentialWriterOption defaultWriterOption = SequentialWriterOption.newBuilder() .trickleFsync(DatabaseDescriptor.getTrickleFsync()) .trickleFsyncByteInterval(DatabaseDescriptor.getTrickleFsyncIntervalInKb() * 1024) .bufferType(BufferType.OFF_HEAP) diff --git a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java index 2bf5a4f0af00..e76f17912499 100644 --- a/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java +++ b/src/java/org/apache/cassandra/index/sai/utils/PrimaryKey.java @@ -27,6 +27,7 @@ import org.apache.cassandra.index.sai.disk.format.IndexFeatureSet; import org.apache.cassandra.index.sai.disk.v1.PartitionAwarePrimaryKeyFactory; import org.apache.cassandra.index.sai.disk.v2.RowAwarePrimaryKeyFactory; +import org.apache.cassandra.index.sai.disk.v2.TokenOnlyPrimaryKey; import org.apache.cassandra.utils.bytecomparable.ByteComparable; import org.apache.cassandra.utils.bytecomparable.ByteSource; @@ -39,7 +40,7 @@ * For the V2 on-disk format the {@link DecoratedKey} and {@link Clustering} are supported. * */ -public interface PrimaryKey extends Comparable, Accountable +public interface PrimaryKey extends Comparable, Accountable, ByteComparable { /** * A factory for creating {@link PrimaryKey} instances @@ -48,14 +49,18 @@ interface Factory { /** * Creates a {@link PrimaryKey} that is represented by a {@link Token}. - * + *

* {@link Token} only primary keys are used for defining the partition range * of a query. * * @param token the {@link Token} * @return a {@link PrimaryKey} represented by a token only */ - PrimaryKey createTokenOnly(Token token); + default PrimaryKey createTokenOnly(Token token) + { + assert token != null; + return new TokenOnlyPrimaryKey(token); + } /** * Creates a {@link PrimaryKey} that is represented by a {@link DecoratedKey}. @@ -118,6 +123,11 @@ static Factory factory(ClusteringComparator clusteringComparator, IndexFeatureSe */ PrimaryKey forStaticRow(); + default boolean isTokenOnly() + { + return false; + } + /** * Returns the {@link Token} associated with this primary key. * @@ -146,6 +156,7 @@ static Factory factory(ClusteringComparator clusteringComparator, IndexFeatureSe * * @return {@code true} if the clustering is empty, otherwise {@code false} */ + @SuppressWarnings("ConstantConditions") default boolean hasEmptyClustering() { return clustering() == null || clustering().isEmpty(); diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java index 08211c0f37ad..eab2da58c6e6 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTable.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java @@ -95,6 +95,7 @@ public abstract class SSTable protected final DiskOptimizationStrategy optimizationStrategy; protected final TableMetadataRef metadata; + public final boolean hasClustering; // This field is null if the compaction metadata is not loaded yet, it can be a empty optional if the compaction metadata is not available protected Optional compactionMetadata; private static final int SAMPLES_CAP = 10000; @@ -111,6 +112,7 @@ protected SSTable(Descriptor descriptor, Set components, TableMetadat this.compression = components.contains(Component.COMPRESSION_INFO); this.components = ImmutableSet.copyOf(components); this.metadata = metadata; + this.hasClustering = metadata().comparator.size() > 0; this.optimizationStrategy = Objects.requireNonNull(optimizationStrategy); } @@ -341,7 +343,7 @@ public long bytesOnDisk() @Override public String toString() { - return getClass().getSimpleName() + "(" + + return getClass().getSimpleName() + '(' + "path='" + getFilename() + '\'' + ')'; } @@ -403,7 +405,7 @@ private static void rewriteTOC(Descriptor descriptor, Collection comp */ public static void writeTOC(File tocFile, Collection components, File.WriteMode writeMode) { - FileOutputStreamPlus fos = null; + FileOutputStreamPlus fos; try (PrintWriter w = new PrintWriter((fos = tocFile.newOutputStream(writeMode)))) { for (Component component : components) @@ -420,7 +422,6 @@ public static void writeTOC(File tocFile, Collection components, File /** * Appends new component names to the TOC component. */ - @SuppressWarnings("resource") protected static void appendTOC(Descriptor descriptor, Collection components) { File tocFile = descriptor.fileFor(Component.TOC); diff --git a/src/java/org/apache/cassandra/io/storage/StorageProvider.java b/src/java/org/apache/cassandra/io/storage/StorageProvider.java index 5916edddf161..757c5b6ec4da 100644 --- a/src/java/org/apache/cassandra/io/storage/StorageProvider.java +++ b/src/java/org/apache/cassandra/io/storage/StorageProvider.java @@ -328,8 +328,8 @@ public FileHandle.Builder fileHandleBuilderFor(IndexComponent.ForRead component) // (and we already have shortcut code for the common 1:1 vector case) // so we leave it alone here if (component.componentType() == IndexComponentType.TERMS_DATA - || component.componentType() == IndexComponentType.VECTOR - || component.componentType() == IndexComponentType.PRIMARY_KEY_TRIE) + || component.componentType() == IndexComponentType.VECTOR) +// || component.componentType() == IndexComponentType.PRIMARY_KEY_TRIE) { builder = builder.adviseRandom(); } diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/QuerySelectivityBench.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/QuerySelectivityBench.java deleted file mode 100644 index 96d5f6076269..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/QuerySelectivityBench.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Copyright DataStax, Inc. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai; - - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Random; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.CassandraRelevantProperties; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.CQLTester; -import org.apache.cassandra.db.ColumnFamilyStore; -import org.apache.cassandra.db.Keyspace; -import org.apache.cassandra.db.commitlog.CommitLog; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; - -/** - * Benchmarks SAI queries with different selectivities. - */ -@BenchmarkMode(Mode.Throughput) -@OutputTimeUnit(TimeUnit.SECONDS) -@Warmup(iterations = 2, time = 2) // seconds -@Measurement(iterations = 5, time = 2) // seconds -@Fork(value = 1) -@Threads(1) -@State(Scope.Benchmark) -public class QuerySelectivityBench extends CQLTester -{ - static final Random RANDOM = new Random(); - - /** The SAI index format version, {@code none} for no index. */ - @Param({ "aa", "ec", "none" }) - public String version; - - /** The number of partitions to be inserted. */ - @Param({ "100" }) - public int partitions; - - /** The number of rows per partition to be inserted. */ - @Param({ "1000" }) - public int rowsPerPartition; - - /** The size of payload added to the clustering key, in bytes. */ - @Param({ "100" }) - public int clusteringSizeInBytes; - - /** The size of payload added to the row's regular columns, in bytes. */ - @Param({ "1000" }) - public int payloadSizeInBytes; - - /** The selectivity of the query, defined as the number of rows matching the query divided by the total number of rows. */ - @Param({ "0.01", "0.03", "0.05", "0.1", "0.2", "0.3", "0.4", "0.5", "0.7", "0.9", "1" }) - public float selectivity; - - /** The LIMIT of the query. */ - @Param({ "5000" }) - public int limit; - - private String partitionQuery; - private String rangeQuery; - - @Setup(Level.Trial) - public void setup() throws Throwable - { - if (!version.equals("none")) - CassandraRelevantProperties.SAI_CURRENT_VERSION.setString(version); - - CQLTester.setUpClass(); - CQLTester.prepareServer(); - beforeTest(); - DatabaseDescriptor.setAutoSnapshot(false); - - // create the schema - String table = createTable("CREATE TABLE %s (k int, c1 int, c2 blob, v int, p blob, PRIMARY KEY (k, c1, c2))"); - if (!version.equals("none")) - createIndex("CREATE CUSTOM INDEX idx ON %s(v) USING 'StorageAttachedIndex'"); - - // disable autocompaction so it doesn't interfere - ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(table); - cfs.disableAutoCompaction(); - - // insert data - for (int k = 0; k < partitions; k++) - { - for (int c1 = 0; c1 < rowsPerPartition; c1++) - { - ByteBuffer c2 = randomBuffer(clusteringSizeInBytes); - int value = RANDOM.nextInt((int) (1.0f / selectivity)); - ByteBuffer payload = randomBuffer(payloadSizeInBytes); - execute("INSERT INTO %s (k, c1, c2, v, p) VALUES (?, ?, ?, ?, ?)", k, c1, c2, value, payload); - } - } - flush(); - - // prepare queries - partitionQuery = String.format("SELECT * FROM %s.%s WHERE k = ? AND v = 0 LIMIT %s ALLOW FILTERING", KEYSPACE, table, limit); - rangeQuery = String.format("SELECT * FROM %s.%s WHERE v = 0 LIMIT %s ALLOW FILTERING", KEYSPACE, table, limit); - } - - @TearDown(Level.Trial) - public void teardown() throws IOException, ExecutionException, InterruptedException - { - CommitLog.instance.shutdownBlocking(); - CQLTester.cleanup(); - } - - @Benchmark - public Object partitionQuery() - { - int k = RANDOM.nextInt(partitions); - return execute(partitionQuery, k).size(); - } - - @Benchmark - public Object rangeQuery() - { - return execute(rangeQuery).size(); - } - - private ByteBuffer randomBuffer(int size) - { - ByteBuffer buffer = ByteBuffer.allocate(size); - RANDOM.nextBytes(buffer.array()); - return buffer; - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/AbstractTrieMemoryIndexBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/AbstractTrieMemoryIndexBenchmark.java deleted file mode 100644 index 697c1fc8089a..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/AbstractTrieMemoryIndexBenchmark.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.memory; - -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; -import java.util.Random; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.cql3.statements.schema.IndexTarget; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.db.marshal.UUIDType; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.index.TargetParser; -import org.apache.cassandra.index.sai.IndexContext; -import org.apache.cassandra.index.sai.StorageAttachedIndex; -import org.apache.cassandra.index.sai.analyzer.NonTokenizingOptions; -import org.apache.cassandra.index.sai.memory.MemoryIndex; -import org.apache.cassandra.schema.CachingParams; -import org.apache.cassandra.schema.ColumnMetadata; -import org.apache.cassandra.schema.IndexMetadata; -import org.apache.cassandra.schema.MockSchema; -import org.apache.cassandra.schema.TableMetadata; -import org.apache.cassandra.utils.Pair; -import org.apache.cassandra.utils.UUIDGen; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Setup; - -public abstract class AbstractTrieMemoryIndexBenchmark -{ - private static final String KEYSPACE = "test_keyspace"; - private static final String TABLE = "test_table"; - private static final String PARTITION_KEY = "key"; - private static final String STRING_COLUMN = "string"; - private static final String STRING_INDEX = "string_index"; - private static final String INTEGER_COLUMN = "integer"; - private static final String INTEGER_INDEX = "integer_index"; - private static final int RANDOM_STRING_SIZE = 64 * 1024 * 1024; - - private char[] randomChars = new char[RANDOM_STRING_SIZE]; - - protected int randomSeed; - - protected IndexContext stringContext; - protected IndexContext integerContext; - - protected MemoryIndex stringIndex; - protected MemoryIndex integerIndex; - - protected ByteBuffer[] stringTerms; - protected ByteBuffer[] integerTerms; - protected DecoratedKey[] partitionKeys; - - @Setup(Level.Trial) - public void initialiseConfig() - { - DatabaseDescriptor.daemonInitialization(); - Random random = new Random(); - randomSeed = random.nextInt(); - for (int i = 0; i < RANDOM_STRING_SIZE; i++) - { - randomChars[i] = (char)('a' + random.nextInt(26)); - } - - ColumnMetadata string = ColumnMetadata.regularColumn(KEYSPACE, TABLE, STRING_COLUMN, UTF8Type.instance); - ColumnMetadata integer = ColumnMetadata.regularColumn(KEYSPACE, TABLE, INTEGER_COLUMN, Int32Type.instance); - TableMetadata table = TableMetadata.builder(KEYSPACE, TABLE) - .addPartitionKeyColumn(PARTITION_KEY, UTF8Type.instance) - .addRegularColumn(STRING_COLUMN, UTF8Type.instance) - .addRegularColumn(INTEGER_COLUMN, Int32Type.instance) - .partitioner(Murmur3Partitioner.instance) - .caching(CachingParams.CACHE_NOTHING) - .build(); - - Map stringOptions = new HashMap<>(); - stringOptions.put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, StorageAttachedIndex.class.getCanonicalName()); - stringOptions.put(NonTokenizingOptions.CASE_SENSITIVE, "true"); - stringOptions.put(IndexTarget.TARGET_OPTION_NAME, STRING_COLUMN); - - Map integerOptions = new HashMap<>(); - integerOptions.put(IndexTarget.CUSTOM_INDEX_OPTION_NAME, StorageAttachedIndex.class.getCanonicalName()); - integerOptions.put(IndexTarget.TARGET_OPTION_NAME, INTEGER_COLUMN); - - IndexMetadata stringMetadata = IndexMetadata.fromSchemaMetadata(STRING_INDEX, IndexMetadata.Kind.CUSTOM, stringOptions); - Pair target = TargetParser.parse(table, stringMetadata); - stringContext = new IndexContext(table.keyspace, - table.name, - table.id, - table.partitionKeyType, - table.comparator, - target.left, - target.right, - stringMetadata, - MockSchema.newCFS(table)); - - IndexMetadata integerMetadata = IndexMetadata.fromSchemaMetadata(INTEGER_INDEX, IndexMetadata.Kind.CUSTOM, integerOptions); - integerContext = new IndexContext(table.keyspace, - table.name, - table.id, - table.partitionKeyType, - table.comparator, - target.left, - target.right, - integerMetadata, - MockSchema.newCFS(table)); - } - - - protected void initialiseColumnData(int numberOfTerms, int rowsPerPartition) - { - Random random = new Random(randomSeed); - - int numberOfKeys = numberOfTerms / rowsPerPartition; - stringTerms = new ByteBuffer[numberOfTerms]; - integerTerms = new ByteBuffer[numberOfTerms]; - partitionKeys = new DecoratedKey[numberOfKeys]; - - int length = 64; - - for (int i = 0; i < numberOfTerms; i++) - { - stringTerms[i] = UTF8Type.instance.decompose(generateRandomString(random, length)); - integerTerms[i] = Int32Type.instance.decompose(i); - } - - for (int i = 0; i < numberOfKeys; i++) - { - partitionKeys[i] = Murmur3Partitioner.instance.decorateKey(UUIDType.instance.decompose(UUIDGen.getTimeUUID())); - } - } - - private String generateRandomString(Random random, int length) - { - return new String(randomChars, random.nextInt(RANDOM_STRING_SIZE - length), length); - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/ReadTrieMemoryIndexBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/ReadTrieMemoryIndexBenchmark.java deleted file mode 100644 index 04de77d73440..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/ReadTrieMemoryIndexBenchmark.java +++ /dev/null @@ -1,141 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.memory; - -import java.util.Random; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.cql3.Operator; -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.DataRange; -import org.apache.cassandra.db.PartitionPosition; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.dht.AbstractBounds; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.index.sai.memory.TrieMemoryIndex; -import org.apache.cassandra.index.sai.plan.Expression; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; - -@Fork(1) -@Warmup(iterations = 5, time = 3) -@Measurement(iterations = 10, time = 3) -@OutputTimeUnit(TimeUnit.MICROSECONDS) -@BenchmarkMode(Mode.AverageTime) -@State(Scope.Thread) -public class ReadTrieMemoryIndexBenchmark extends AbstractTrieMemoryIndexBenchmark -{ - private static final int NUMBER_OF_SEARCHES = 1000; - private static final AbstractBounds ALL_DATA_RANGE = DataRange.allData(Murmur3Partitioner.instance).keyRange(); - - @Param({ "1000", "10000", "100000", "1000000" }) - protected int numberOfTerms; - - @Param({ "1", "10", "100"}) - protected int rowsPerPartition; - - private Random random; - private Expression[] stringEqualityExpressions; - private Expression[] integerEqualityExpressions; - private Expression[] integerRangeExpressions; - - @Setup(Level.Iteration) - public void initialiseIndexes() - { - initialiseColumnData(numberOfTerms, rowsPerPartition); - stringIndex = new TrieMemoryIndex(stringContext); - integerIndex = new TrieMemoryIndex(integerContext); - - int rowCount = 0; - int keyCount = 0; - for (int i = 0; i < numberOfTerms; i++) - { - stringIndex.add(partitionKeys[keyCount], Clustering.EMPTY, stringTerms[i], allocatedBytes -> {}, allocatesBytes -> {}); - integerIndex.add(partitionKeys[keyCount], Clustering.EMPTY, integerTerms[i], allocatedBytes -> {}, allocatesBytes -> {}); - if (++rowCount == rowsPerPartition) - { - rowCount = 0; - keyCount++; - } - } - random = new Random(randomSeed); - - stringEqualityExpressions = new Expression[NUMBER_OF_SEARCHES]; - integerEqualityExpressions = new Expression[NUMBER_OF_SEARCHES]; - integerRangeExpressions = new Expression[NUMBER_OF_SEARCHES]; - - for (int i = 0; i < NUMBER_OF_SEARCHES; i++) - { - stringEqualityExpressions[i] = new Expression(stringContext).add(Operator.EQ, stringTerms[random.nextInt(numberOfTerms)]); - integerEqualityExpressions[i] = new Expression(integerContext).add(Operator.EQ, integerTerms[random.nextInt(numberOfTerms)]); - - int lowerValue = random.nextInt(numberOfTerms - 10); - - integerRangeExpressions[i] = new Expression(integerContext) - {{ - operation = Op.RANGE; - lower = new Bound(Int32Type.instance.decompose(lowerValue), Int32Type.instance, true); - upper = new Bound(Int32Type.instance.decompose(lowerValue + 10), Int32Type.instance, true); - }}; - } - } - - @Benchmark - public long stringEqualityBenchmark() - { - long size = 0; - for (int i = 0; i < NUMBER_OF_SEARCHES; i++) - { - stringIndex.search(stringEqualityExpressions[i], ALL_DATA_RANGE); - } - return size; - } - - @Benchmark - public long integerEqualityBenchmark() - { - long size = 0; - for (int i = 0; i < NUMBER_OF_SEARCHES; i++) - { - integerIndex.search(integerEqualityExpressions[i], ALL_DATA_RANGE); - } - return size; - } - - @Benchmark - public long integerRangeBenchmark() - { - long size = 0; - for (int i = 0; i < NUMBER_OF_SEARCHES; i++) - { - integerIndex.search(integerRangeExpressions[i], ALL_DATA_RANGE); - } - return size; - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/WriteTrieMemoryIndexBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/WriteTrieMemoryIndexBenchmark.java deleted file mode 100644 index 25a4e8d31f05..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/memory/WriteTrieMemoryIndexBenchmark.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.memory; - - -import java.nio.ByteBuffer; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.index.sai.memory.TrieMemoryIndex; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; - -@Fork(1) -@Warmup(iterations = 5, time = 3) -@Measurement(iterations = 10, time = 3) -@OutputTimeUnit(TimeUnit.MICROSECONDS) -@BenchmarkMode(Mode.AverageTime) -@State(Scope.Thread) -public class WriteTrieMemoryIndexBenchmark extends AbstractTrieMemoryIndexBenchmark -{ - @Param({ "1000", "10000", "100000", "1000000" }) - protected int numberOfTerms; - - @Param({ "1", "10", "100"}) - protected int rowsPerPartition; - - @Setup(Level.Iteration) - public void initialiseColumnData() - { - initialiseColumnData(numberOfTerms, rowsPerPartition); - } - - @Setup(Level.Invocation) - public void initialiseIndexes() - { - stringIndex = new TrieMemoryIndex(stringContext); - integerIndex = new TrieMemoryIndex(integerContext); - } - - @Benchmark - public long writeStringIndex() - { - long size = 0; - int rowCount = 0; - int keyCount = 0; - for (ByteBuffer term : stringTerms) - { - stringIndex.add(partitionKeys[keyCount], Clustering.EMPTY, term, allocatedBytes -> {}, allocatesBytes -> {}); - if (++rowCount == rowsPerPartition) - { - rowCount = 0; - keyCount++; - } - size++; - } - return size; - } - - @Benchmark - public long writeIntegerIndex() - { - long size = 0; - int rowCount = 0; - int keyCount = 0; - for (ByteBuffer term : integerTerms) - { - integerIndex.add(partitionKeys[keyCount], Clustering.EMPTY, term, allocatedBytes -> {}, allocatesBytes -> {}); - if (++rowCount == rowsPerPartition) - { - rowCount = 0; - keyCount++; - } - size++; - } - return size; - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/AbstractOnDiskBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/AbstractOnDiskBenchmark.java deleted file mode 100644 index 8f7bb85883a0..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/AbstractOnDiskBenchmark.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.v1; - -import java.io.IOException; -import java.nio.file.Files; -import java.util.Random; -import java.util.stream.IntStream; - -import com.google.common.base.Stopwatch; - -import org.apache.cassandra.Util; -import org.apache.cassandra.cache.ChunkCache; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.marshal.IntegerType; -import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.index.sai.IndexContext; -import org.apache.cassandra.index.sai.SAITester; -import org.apache.cassandra.index.sai.disk.format.IndexComponents; -import org.apache.cassandra.index.sai.disk.format.IndexComponentType; -import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; -import org.apache.cassandra.index.sai.disk.io.IndexInput; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; -import org.apache.cassandra.index.sai.disk.v1.SSTableComponentsWriter; -import org.apache.cassandra.index.sai.disk.v1.bitpack.BlockPackedReader; -import org.apache.cassandra.index.sai.disk.v1.MetadataSource; -import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader; -import org.apache.cassandra.index.sai.disk.v1.postings.PostingsWriter; -import org.apache.cassandra.index.sai.metrics.QueryEventListener; -import org.apache.cassandra.index.sai.postings.IntArrayPostingList; -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.apache.cassandra.index.sai.utils.IndexFileUtils; -import org.apache.cassandra.io.sstable.Descriptor; -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.schema.TableMetadata; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.TearDown; - -public abstract class AbstractOnDiskBenchmark -{ - private static Random random = new Random(); - - protected TableMetadata metadata; - protected IndexDescriptor indexDescriptor; - - private Descriptor descriptor; - - String index; - IndexContext indexContext; - private FileHandle token; - private FileHandle postings; - private long summaryPosition; - - /** - * @return num of rows to be stored in per-sstable components - */ - public abstract int numRows(); - - /** - * @return num of postings to be written in posting file - */ - public abstract int numPostings(); - - /** - * To be called before executing each @Benchmark method - */ - public abstract void beforeInvocation() throws Throwable; - - /** - * To be called after executing each @Benchmark method - */ - public abstract void afterInvocation() throws Throwable; - - protected int toPosting(int id) - { - return id; - } - - protected long toToken(long id) - { - return id * 16_013L + random.nextInt(16_000); - } - - protected long toOffset(long id) - { - return id * 16_013L + random.nextInt(16_000); - } - - @Setup(Level.Trial) - public void perTrialSetup() throws IOException - { - DatabaseDescriptor.daemonInitialization(); // required to use ChunkCache - assert ChunkCache.instance != null; - - String keyspaceName = "ks"; - String tableName = this.getClass().getSimpleName(); - metadata = TableMetadata - .builder(keyspaceName, tableName) - .partitioner(Murmur3Partitioner.instance) - .addPartitionKeyColumn("pk", UTF8Type.instance) - .addRegularColumn("col", IntegerType.instance) - .build(); - - descriptor = new Descriptor(new File(Files.createTempDirectory("jmh").toFile()), - metadata.keyspace, - metadata.name, - Util.newUUIDGen().get()); - indexDescriptor = IndexDescriptor.empty(descriptor); - index = "test"; - indexContext = SAITester.createIndexContext(index, IntegerType.instance); - - // write per-sstable components: token and offset - writeSSTableComponents(numRows()); - token = indexDescriptor.perSSTableComponents().get(IndexComponentType.TOKEN_VALUES).createFileHandle(); - - // write postings - summaryPosition = writePostings(numPostings()); - postings = indexDescriptor.perIndexComponents(indexContext).get(IndexComponentType.POSTING_LISTS).createFileHandle(); - } - - @TearDown(Level.Trial) - public void perTrialTearDown() - { - token.close(); - postings.close(); - FileUtils.deleteRecursive(descriptor.directory); - } - - @Setup(Level.Invocation) - public void perInvocationSetup() throws Throwable - { - beforeInvocation(); - } - - @TearDown(Level.Invocation) - public void perInvocationTearDown() throws Throwable - { - afterInvocation(); - } - - private long writePostings(int rows) throws IOException - { - final int[] postings = IntStream.range(0, rows).map(this::toPosting).toArray(); - final IntArrayPostingList postingList = new IntArrayPostingList(postings); - - try (PostingsWriter writer = new PostingsWriter(indexDescriptor.newPerIndexComponentsForWrite(indexContext))) - { - long summaryPosition = writer.write(postingList); - writer.complete(); - - return summaryPosition; - } - } - - protected final PostingsReader openPostingsReader() throws IOException - { - IndexInput input = IndexFileUtils.instance.openInput(postings); - IndexInput summaryInput = IndexFileUtils.instance.openInput(postings); - - PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(summaryInput, summaryPosition); - return new PostingsReader(input, summary, QueryEventListener.PostingListEventListener.NO_OP); - } - - private void writeSSTableComponents(int rows) throws IOException - { - SSTableComponentsWriter writer = new SSTableComponentsWriter(indexDescriptor.newPerSSTableComponentsForWrite()); - for (int i = 0; i < rows; i++) - writer.recordCurrentTokenOffset(toToken(i), toOffset(i)); - - writer.complete(Stopwatch.createStarted()); - } - - protected final LongArray openRowIdToTokenReader() throws IOException - { - IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); - MetadataSource source = MetadataSource.loadMetadata(components); - NumericValuesMeta tokensMeta = new NumericValuesMeta(source.get(components.get(IndexComponentType.TOKEN_VALUES))); - return new BlockPackedReader(token, tokensMeta).open(); - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/BlockPackedReaderBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/BlockPackedReaderBenchmark.java deleted file mode 100644 index 5f27d6d1f959..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/BlockPackedReaderBenchmark.java +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.v1; - -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OperationsPerInvocation; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(value = 1, jvmArgsAppend = { - // "-XX:+UnlockCommercialFeatures", "-XX:+FlightRecorder","-XX:+UnlockDiagnosticVMOptions", "-XX:+DebugNonSafepoints", - // "-XX:StartFlightRecording=duration=60s,filename=./BlockPackedReaderBenchmark.jfr,name=profile,settings=profile", - // "-XX:FlightRecorderOptions=settings=/home/jake/workspace/cassandra/profiling-advanced.jfc,samplethreads=true" -}) -@Warmup(iterations = 3) -@Measurement(iterations = 5, timeUnit = TimeUnit.NANOSECONDS) -@OutputTimeUnit(TimeUnit.NANOSECONDS) -@State(Scope.Thread) -public class BlockPackedReaderBenchmark extends AbstractOnDiskBenchmark -{ - private static final int NUM_INVOCATIONS = 10_000; - - @Param({ "1", "10", "100", "1000"}) - public int skippingDistance; - - protected LongArray rowIdToToken; - private int[] rowIds; - private long[] tokenValues; - - @Override - public int numRows() - { - return 10_000_000; - } - - @Override - public int numPostings() - { - return 10_000_000; - } - - @Override - public void beforeInvocation() throws Throwable - { - // rowIdToToken.findTokenRowID keeps track of last position, so it must be per-benchmark-method-invocation. - rowIdToToken = openRowIdToTokenReader(); - - rowIds = new int[NUM_INVOCATIONS]; - tokenValues = new long[NUM_INVOCATIONS]; - - for (int i = 0; i < rowIds.length; i++) - { - rowIds[i] = toPosting(i * skippingDistance); - tokenValues[i] = toToken(rowIds[i]); - } - } - - @Override - public void afterInvocation() throws Throwable - { - rowIdToToken.close(); - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput, Mode.AverageTime }) - public void get(Blackhole bh) - { - for (int i = 0; i < rowIds.length;) - { - bh.consume(rowIdToToken.get(rowIds[i])); - i++; - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput, Mode.AverageTime }) - public void findTokenRowID(Blackhole bh) - { - for (int i = 0; i < tokenValues.length;) - { - bh.consume(rowIdToToken.ceilingRowId(tokenValues[i])); - i++; - } - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/MergePostingListBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/MergePostingListBenchmark.java deleted file mode 100644 index 98429b803ff0..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/MergePostingListBenchmark.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.v1; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import org.apache.cassandra.index.sai.disk.PostingList; -import org.apache.cassandra.index.sai.disk.v1.postings.MergePostingList; -import org.apache.cassandra.index.sai.postings.IntArrayPostingList; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(1) -@Warmup(iterations = 3) -@Measurement(iterations = 5) -@State(Scope.Thread) -public class MergePostingListBenchmark -{ - @Param({"3", "17"}) - int stepMax = 7; - - @Param({"50", "5000"}) - int sources = 50; - - @Param({"1000000"}) - int count = 1_000_000; - - @Param({"UNIFORM", "NORMAL", "SEQUENTIAL", "ROUND_ROBIN"}) - Distribution distribution = Distribution.NORMAL; - - public enum Distribution - { - UNIFORM, NORMAL, SEQUENTIAL, ROUND_ROBIN - } - - List splitPostingLists = new ArrayList<>(); - PostingList merge; - - @Setup(Level.Trial) - public void generatePostings() - { - final AtomicInteger rowId = new AtomicInteger(); - final Random rand = new Random(1); - final int[] postings = IntStream.generate(() -> rowId.addAndGet(rand.nextInt(stepMax))) - .limit(count) - .toArray(); - - // split postings into multiple lists - Function grouping; - switch (distribution) - { - case UNIFORM: - grouping = x -> rand.nextInt(sources); - break; - case NORMAL: - grouping = x -> (int) Math.min(sources - 1, Math.abs(rand.nextGaussian()) * sources / 5); - break; - case SEQUENTIAL: - { - AtomicInteger index = new AtomicInteger(); - int sizePerList = Math.max(count / sources, 1); - grouping = x -> index.getAndIncrement() / sizePerList; - break; - } - case ROUND_ROBIN: - { - AtomicInteger index = new AtomicInteger(); - grouping = x -> index.getAndIncrement() % sources; - break; - } - default: - throw new AssertionError(); - } - final Map> splitPostings = Arrays.stream(postings) - .boxed() - .collect(Collectors.groupingBy(grouping)); - - for (List split : splitPostings.values()) - { - // Remove any duplicates in each individual set - int[] data = split.stream().distinct().mapToInt(Integer::intValue).toArray(); - splitPostingLists.add(data); - } - } - - @Setup(Level.Invocation) - public void mergePostings() - { - var lists = new ArrayList(); - for (int[] postings : splitPostingLists) - { - lists.add(new IntArrayPostingList(postings)); - } - merge = MergePostingList.merge(lists); - } - - @Benchmark - @BenchmarkMode({ Mode.AverageTime }) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void nextPostingIteration(Blackhole bh) throws IOException - { - long id; - while ((id = merge.nextPosting()) != PostingList.END_OF_STREAM) - { - bh.consume(id); - } - } - - @Benchmark - @BenchmarkMode({ Mode.AverageTime }) - @OutputTimeUnit(TimeUnit.MILLISECONDS) - public void advanceIteration(Blackhole bh) throws IOException - { - int id = 0; - while ((id = merge.advance(id + stepMax)) != PostingList.END_OF_STREAM) - { - bh.consume(id); - } - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/PostingsReaderBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/PostingsReaderBenchmark.java deleted file mode 100644 index 0eb0fb3d5368..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v1/PostingsReaderBenchmark.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.v1; - -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader; -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OperationsPerInvocation; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(1) -@Warmup(iterations = 3) -@Measurement(iterations = 5, timeUnit = TimeUnit.MILLISECONDS) -@OutputTimeUnit(TimeUnit.MICROSECONDS) -@State(Scope.Thread) -public class PostingsReaderBenchmark extends AbstractOnDiskBenchmark -{ - private static final int NUM_INVOCATIONS = 10_000; - - @Param({ "1", "10", "100", "1000"}) - public int skippingDistance; - - protected LongArray rowIdToToken; - protected PostingsReader reader; - private int[] rowIds; - protected long[] tokenValues; - - @Override - public int numRows() - { - return 10_000_000; - } - - @Override - public int numPostings() - { - return 10_000_000; - } - - @Override - public void beforeInvocation() throws Throwable - { - // rowIdToToken.findTokenRowID keeps track of last position, so it must be per-benchmark-method-invocation. - rowIdToToken = openRowIdToTokenReader(); - reader = openPostingsReader(); - - tokenValues = new long[NUM_INVOCATIONS]; - rowIds = new int[NUM_INVOCATIONS]; - for (int i = 0; i < tokenValues.length; i++) - { - rowIds[i] = toPosting(i * skippingDistance); - tokenValues[i] = toToken(i * skippingDistance); - } - } - - @Override - public void afterInvocation() throws Throwable - { - rowIdToToken.close(); - reader.close(); - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput, Mode.AverageTime }) - public void skipAndRequestNext(Blackhole bh) throws Throwable - { - int rowId = -1; - for (int i = 0; i < tokenValues.length;) - { - long token = tokenValues[i]; - if (rowId < 0) - rowId = (int) rowIdToToken.ceilingRowId(token); - bh.consume(reader.advance(rowId)); - rowId = -1; - - i++; - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput, Mode.AverageTime }) - public void advance(Blackhole bh) throws Throwable - { - for (int i = 0; i < tokenValues.length;) - { - int rowId = rowIds[i]; - bh.consume(reader.advance(rowId)); - - i++; - } - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v2/sortedbytes/SortedTermsBenchmark.java b/test/microbench/org/apache/cassandra/test/microbench/index/sai/v2/sortedbytes/SortedTermsBenchmark.java deleted file mode 100644 index 4db530aa0c3e..000000000000 --- a/test/microbench/org/apache/cassandra/test/microbench/index/sai/v2/sortedbytes/SortedTermsBenchmark.java +++ /dev/null @@ -1,287 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.test.microbench.index.sai.v2.sortedbytes; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.index.sai.disk.format.IndexComponent; -import org.apache.cassandra.index.sai.disk.format.IndexComponentType; -import org.apache.cassandra.index.sai.disk.format.IndexComponents; -import org.apache.cassandra.index.sai.disk.v1.LongArray; -import org.apache.cassandra.index.sai.disk.v1.MetadataSource; -import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsMeta; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsReader; -import org.apache.cassandra.index.sai.disk.v2.sortedterms.SortedTermsWriter; -import org.apache.cassandra.index.sai.utils.TypeUtil; -import org.apache.cassandra.io.util.FileHandle; -import org.apache.cassandra.test.microbench.index.sai.v1.AbstractOnDiskBenchmark; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; -import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; -import org.apache.lucene.analysis.core.WhitespaceAnalyzer; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.SortedDocValuesField; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedDocValues; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FSDirectory; -import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.NumericUtils; -import org.openjdk.jmh.annotations.Benchmark; -import org.openjdk.jmh.annotations.BenchmarkMode; -import org.openjdk.jmh.annotations.Fork; -import org.openjdk.jmh.annotations.Level; -import org.openjdk.jmh.annotations.Measurement; -import org.openjdk.jmh.annotations.Mode; -import org.openjdk.jmh.annotations.OperationsPerInvocation; -import org.openjdk.jmh.annotations.OutputTimeUnit; -import org.openjdk.jmh.annotations.Param; -import org.openjdk.jmh.annotations.Scope; -import org.openjdk.jmh.annotations.Setup; -import org.openjdk.jmh.annotations.State; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(value = 1, jvmArgsAppend = { - // "-XX:+UnlockCommercialFeatures", "-XX:+FlightRecorder","-XX:+UnlockDiagnosticVMOptions", "-XX:+DebugNonSafepoints", - // "-XX:StartFlightRecording=duration=60s,filename=./BlockPackedReaderBenchmark.jfr,name=profile,settings=profile", - // "-XX:FlightRecorderOptions=settings=/home/jake/workspace/cassandra/profiling-advanced.jfc,samplethreads=true" -}) -@Warmup(iterations = 1) -@Measurement(iterations = 1, timeUnit = TimeUnit.MICROSECONDS) -@OutputTimeUnit(TimeUnit.MICROSECONDS) -@State(Scope.Thread) -public class SortedTermsBenchmark extends AbstractOnDiskBenchmark -{ - private static final int NUM_ROWS = 1_000_000; - private static final int NUM_INVOCATIONS = 1_000; // must be <= (NUM_ROWS / max(skippingDistance)) - - @Param({ "1", "10", "100", "1000"}) - public int skippingDistance; - - protected LongArray rowIdToToken; - private int[] rowIds; - private long[] tokenValues; - FileHandle trieFile; - FileHandle termsData; - FileHandle blockOffsets; - SortedTermsReader sortedTermsReader; - Path luceneDir; - Directory directory; - DirectoryReader luceneReader; - SortedDocValues columnASortedDocValues; - - @Override - public int numRows() - { - return NUM_ROWS; - } - - @Override - public int numPostings() - { - return NUM_ROWS; - } - - byte[][] bcIntBytes = new byte[NUM_ROWS][]; - - @Setup(Level.Trial) - public void perTrialSetup2() throws IOException - { - IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); - try (MetadataWriter metadataWriter = new MetadataWriter(components); - NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) - { - for (int x = 0; x < NUM_ROWS; x++) - { - ByteBuffer buffer = Int32Type.instance.decompose(x); - ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, TypeUtil.BYTE_COMPARABLE_VERSION); - byte[] bytes = ByteSourceInverse.readBytes(byteSource); - bcIntBytes[x] = bytes; - writer.add(ByteComparable.preencoded(TypeUtil.BYTE_COMPARABLE_VERSION, bytes)); - } - } - - // create the lucene index - luceneDir = Files.createTempDirectory("jmh_lucene_test"); - directory = FSDirectory.open(luceneDir); - IndexWriterConfig config = new IndexWriterConfig(new WhitespaceAnalyzer()); - IndexWriter indexWriter = new IndexWriter(directory, config); - - Document document = new Document(); - - int i = 0; - for (int x = 0; x < NUM_ROWS; x++) - { - document.clear(); - byte[] bytes = new byte[4]; - NumericUtils.intToSortableBytes(x, bytes, 0); - document.add(new SortedDocValuesField("columnA", new BytesRef(bytes))); - indexWriter.addDocument(document); - luceneBytes[x] = bytes; - } - indexWriter.forceMerge(1); - indexWriter.close(); - } - - byte[][] luceneBytes = new byte[NUM_ROWS][]; - - @Override - public void beforeInvocation() throws Throwable - { - // rowIdToToken.findTokenRowID keeps track of last position, so it must be per-benchmark-method-invocation. - rowIdToToken = openRowIdToTokenReader(); - - rowIds = new int[NUM_ROWS]; - tokenValues = new long[NUM_ROWS]; - - IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); - MetadataSource metadataSource = MetadataSource.loadMetadata(components); - IndexComponent.ForRead blocksComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCKS); - IndexComponent.ForRead blockOffsetsComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS); - NumericValuesMeta blockOffsetMeta = new NumericValuesMeta(metadataSource.get(blockOffsetsComponent)); - SortedTermsMeta sortedTermsMeta = new SortedTermsMeta(metadataSource.get(blocksComponent)); - trieFile = components.get(IndexComponentType.PRIMARY_KEY_TRIE).createFileHandle(); - termsData = blocksComponent.createFileHandle(); - blockOffsets = blockOffsetsComponent.createFileHandle(); - - sortedTermsReader = new SortedTermsReader(termsData,blockOffsets, trieFile, sortedTermsMeta, blockOffsetMeta); - - luceneReader = DirectoryReader.open(directory); - LeafReaderContext context = luceneReader.leaves().get(0); - - columnASortedDocValues = context.reader().getSortedDocValues("columnA"); - } - - @Override - public void afterInvocation() throws Throwable - { - luceneReader.close(); - termsData.close(); - blockOffsets.close(); - rowIdToToken.close(); - trieFile.close(); - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void luceneSeekToPointID(Blackhole bh) throws IOException - { - for (int i = 0; i < NUM_INVOCATIONS;) - { - bh.consume(columnASortedDocValues.lookupOrd(i)); - i += skippingDistance; - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void luceneSeekToTerm(Blackhole bh) throws IOException - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - bh.consume(columnASortedDocValues.lookupTerm(new BytesRef(luceneBytes[i * skippingDistance]))); - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void advance(Blackhole bh) throws IOException - { - try (SortedTermsReader.Cursor cursor = sortedTermsReader.openCursor()) - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - cursor.advance(); - bh.consume(cursor.term()); - } - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void seekToPointID(Blackhole bh) throws IOException - { - try (SortedTermsReader.Cursor cursor = sortedTermsReader.openCursor()) - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - cursor.seekToPointId((long) i * skippingDistance); - bh.consume(cursor.term()); - } - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void seekToTerm(Blackhole bh) throws IOException - { - try (SortedTermsReader.Cursor cursor = sortedTermsReader.openCursor()) - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - int iFinal = i; - bh.consume(cursor.ceiling(v -> ByteSource.preencoded(this.bcIntBytes[iFinal * skippingDistance]))); - } - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void get(Blackhole bh) - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - bh.consume(rowIdToToken.get(rowIds[i * skippingDistance])); - } - } - - @Benchmark - @OperationsPerInvocation(NUM_INVOCATIONS) - @BenchmarkMode({ Mode.Throughput}) - public void longArrayFindTokenRowID(Blackhole bh) - { - for (int i = 0; i < NUM_INVOCATIONS; i++) - { - bh.consume(rowIdToToken.ceilingRowId(tokenValues[i * skippingDistance])); - } - } -} diff --git a/test/microbench/org/apache/cassandra/test/microbench/sai/KeyLookupBench.java b/test/microbench/org/apache/cassandra/test/microbench/sai/KeyLookupBench.java new file mode 100644 index 000000000000..528d81a1b97a --- /dev/null +++ b/test/microbench/org/apache/cassandra/test/microbench/sai/KeyLookupBench.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.test.microbench.sai; + +import java.nio.ByteBuffer; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Stopwatch; + +import org.apache.cassandra.Util; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.CompositeType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.index.sai.disk.PerSSTableWriter; +import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; +import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.index.sai.disk.v2.RowAwarePrimaryKeyFactory; +import org.apache.cassandra.index.sai.disk.v2.WidePrimaryKeyMap; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.sstable.Descriptor; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.schema.TableMetadata; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +import static org.apache.cassandra.index.sai.SAITester.getRandom; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@BenchmarkMode({ Mode.Throughput }) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@Warmup(iterations = 3, time = 1) +@Measurement(iterations = 3, time = 5) +@Fork(value = 1, jvmArgsAppend = "-Xmx512M") +@Threads(1) +@State(Scope.Benchmark) +public class KeyLookupBench +{ + private static final int rows = 1_000_000; + + static + { + DatabaseDescriptor.toolInitialization(); + // Partitioner is not set in client mode. + if (DatabaseDescriptor.getPartitioner() == null) + DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance); + } + + protected TableMetadata metadata; + protected IndexDescriptor indexDescriptor; + + private PrimaryKeyMap primaryKeyMap; + + private PrimaryKey primaryKey; + + @Param({ "3", "4", "5" }) + public int partitionBlockShift; + + @Param({ "3", "4", "5" }) + public int clusteringBlockShift; + + @Param({ "10", "100", "1000", "10000" }) + public int partitionSize; + + @Param({ "true", "false" }) + public boolean randomClustering; + + @Setup(Level.Trial) + public void trialSetup() throws Exception + { + String keyspaceName = "ks"; + String tableName = this.getClass().getSimpleName(); + metadata = TableMetadata + .builder(keyspaceName, tableName) + .partitioner(Murmur3Partitioner.instance) + .addPartitionKeyColumn("pk1", LongType.instance) + .addPartitionKeyColumn("pk2", LongType.instance) + .addClusteringColumn("ck1", UTF8Type.instance) + .addClusteringColumn("ck2", UTF8Type.instance) + .build(); + + Descriptor descriptor = new Descriptor(new File(Files.createTempDirectory("jmh").toFile()), + metadata.keyspace, + metadata.name, + Util.newUUIDGen().get()); + + indexDescriptor = IndexDescriptor.empty(descriptor, metadata); + + CassandraRelevantProperties.SAI_SORTED_TERMS_PARTITION_BLOCK_SHIFT.setInt(partitionBlockShift); + CassandraRelevantProperties.SAI_SORTED_TERMS_CLUSTERING_BLOCK_SHIFT.setInt(clusteringBlockShift); + PerSSTableWriter writer = Version.current().onDiskFormat().newPerSSTableWriter(indexDescriptor); +// new SSTableComponentsWriter(indexDescriptor); + + RowAwarePrimaryKeyFactory factory = new RowAwarePrimaryKeyFactory(metadata.comparator); + + PrimaryKey[] primaryKeys = new PrimaryKey[rows]; + int partition = 0; + int partitionRowCounter = 0; + for (int index = 0; index < rows; index++) + { + primaryKeys[index] = factory.create(makeKey(metadata, (long) partition, (long) partition), makeClustering(metadata)); + partitionRowCounter++; + if (partitionRowCounter == partitionSize) + { + partition++; + partitionRowCounter = 0; + } + } + + Arrays.sort(primaryKeys); + + DecoratedKey lastKey = null; + for (PrimaryKey primaryKey : primaryKeys) + { + if (lastKey == null || lastKey.compareTo(primaryKey.partitionKey()) < 0) + { + lastKey = primaryKey.partitionKey(); + writer.startPartition(lastKey); + } + writer.nextRow(primaryKey); + } + + writer.complete(Stopwatch.createStarted()); + + SSTableReader sstableReader = mock(SSTableReader.class); + when(sstableReader.metadata()).thenReturn(metadata); + + PrimaryKeyMap.Factory mapFactory = new WidePrimaryKeyMap.Factory(indexDescriptor.perSSTableComponents(), factory, sstableReader); + + primaryKeyMap = mapFactory.newPerSSTablePrimaryKeyMap(); + + primaryKey = primaryKeys[500000]; + } + + @Benchmark + public long advanceToKey() + { + return primaryKeyMap.rowIdFromPrimaryKey(primaryKey); + } + + private static DecoratedKey makeKey(TableMetadata table, Object... partitionKeys) + { + ByteBuffer key; + if (TypeUtil.isComposite(table.partitionKeyType)) + key = ((CompositeType) table.partitionKeyType).decompose(partitionKeys); + else + key = table.partitionKeyType.fromString((String) partitionKeys[0]); + return table.partitioner.decorateKey(key); + } + + private Clustering makeClustering(TableMetadata table) + { + Clustering clustering; + if (table.comparator.size() == 0) + clustering = Clustering.EMPTY; + else + { + ByteBuffer[] values = new ByteBuffer[table.comparator.size()]; + for (int index = 0; index < table.comparator.size(); index++) + values[index] = table.comparator.subtype(index).fromString(makeClusteringString()); + clustering = Clustering.make(values); + } + return clustering; + } + + private String makeClusteringString() + { + if (randomClustering) + return getRandom().nextTextString(10, 100); + else + return String.format("%08d", getRandom().nextIntBetween(0, partitionSize)); + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/SAITester.java b/test/unit/org/apache/cassandra/index/sai/SAITester.java index cd7bd191eb06..336353f1d136 100644 --- a/test/unit/org/apache/cassandra/index/sai/SAITester.java +++ b/test/unit/org/apache/cassandra/index/sai/SAITester.java @@ -228,7 +228,7 @@ public void corrupt(File file) throws IOException public TestRule testRules = new ResourceLeakDetector(); @Before - public void resetQueryOptimizationLevel() throws Throwable + public void resetQueryOptimizationLevel() { // Enable the optimizer by default. If there are any tests that need to disable it, they can do so explicitly. QueryController.QUERY_OPT_LEVEL = 1; @@ -236,7 +236,7 @@ public void resetQueryOptimizationLevel() throws Throwable } @Before - public void resetLastValidSegmentRowId() throws Throwable + public void resetLastValidSegmentRowId() { // Don't want this setting to impact peer tests SegmentBuilder.updateLastValidSegmentRowId(-1); @@ -346,7 +346,7 @@ protected void simulateNodeRestart(boolean wait) cfs.indexManager.listIndexes().forEach(index -> { ((StorageAttachedIndexGroup)cfs.indexManager.getIndexGroup(index)).reset(); }); - cfs.indexManager.listIndexes().forEach(index -> cfs.indexManager.buildIndex(index)); + cfs.indexManager.listIndexes().forEach(cfs.indexManager::buildIndex); cfs.indexManager.executePreJoinTasksBlocking(true); if (wait) { @@ -572,7 +572,7 @@ protected long getDiskUsage() protected void verifyNoIndexFiles() { - assertTrue(indexFiles().size() == 0); + assertEquals(0, indexFiles().size()); } // Verify every sstables is indexed correctly and the components are valid. @@ -619,7 +619,7 @@ protected void verifyIndexFiles(IndexContext numericIndexContext, { Set indexFiles = indexFiles(); - for (IndexComponentType indexComponentType : Version.current().onDiskFormat().perSSTableComponentTypes()) + for (IndexComponentType indexComponentType : Version.current().onDiskFormat().perSSTableComponentTypes(false)) { Set tableFiles = componentFiles(indexFiles, new Component(Component.Type.CUSTOM, Version.current().fileNameFormatter().format(indexComponentType, (String)null, 0))); assertEquals(tableFiles.toString(), perSSTableFiles, tableFiles.size()); @@ -794,9 +794,9 @@ protected int getCompactionTasks() return CompactionManager.instance.getActiveCompactions() + CompactionManager.instance.getPendingTasks(); } - protected String getSingleTraceStatement(Session session, String query, String contains) throws Throwable + protected String getSingleTraceStatement(Session session, String query, String contains) { - query = String.format(query, KEYSPACE + "." + currentTable()); + query = String.format(query, KEYSPACE + '.' + currentTable()); QueryTrace trace = session.execute(session.prepare(query).bind().enableTracing()).getExecutionInfo().getQueryTrace(); waitForTracingEvents(); @@ -808,7 +808,7 @@ protected String getSingleTraceStatement(Session session, String query, String c return null; } - protected void assertNumRows(int expected, String query, Object... args) throws Throwable + protected void assertNumRows(int expected, String query, Object... args) { ResultSet rs = executeNet(String.format(query, args)); assertEquals(expected, rs.all().size()); @@ -923,7 +923,7 @@ protected static class TestWithConcurrentVerification private final CountDownLatch taskCompleted = new CountDownLatch(1); private final int verificationIntervalInMs; - private final int verificationMaxInMs = 300_000; // 300s + private static final int verificationMaxInMs = 300_000; // 300s public TestWithConcurrentVerification(Runnable verificationTask, Runnable targetTask) { diff --git a/test/unit/org/apache/cassandra/index/sai/cql/NativeIndexDDLTest.java b/test/unit/org/apache/cassandra/index/sai/cql/NativeIndexDDLTest.java index 7f9e1710d02e..7294f7fecb72 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/NativeIndexDDLTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/NativeIndexDDLTest.java @@ -33,7 +33,6 @@ import java.util.stream.LongStream; import org.apache.cassandra.cql3.restrictions.StatementRestrictions; -import org.apache.cassandra.index.sasi.SASIIndex; import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; @@ -1118,7 +1117,7 @@ private void verifyRebuildCorruptedFiles(String numericIndexName, IndexContext numericIndexContext = getIndexContext(numericIndexName); IndexContext stringIndexContext = getIndexContext(stringIndexName); - for (IndexComponentType component : Version.current().onDiskFormat().perSSTableComponentTypes()) + for (IndexComponentType component : Version.current().onDiskFormat().perSSTableComponentTypes(false)) verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, null, corruptionType, true, true, rebuild); for (IndexComponentType component : Version.current().onDiskFormat().perIndexComponentTypes(numericIndexContext)) @@ -1137,12 +1136,14 @@ private void verifyRebuildIndexComponent(IndexContext numericIndexContext, boolean failedNumericIndex, boolean rebuild) throws Throwable { - // The completion markers are valid if they exist on the file system so we only need to test + // The completion markers are valid if they exist on the file system, so we only need to test // their removal. If we are testing with encryption then we don't want to test any components // that are encryptable unless they have been removed because encrypted components aren't // checksum validated. - if (component == IndexComponentType.PRIMARY_KEY_TRIE || component == IndexComponentType.PRIMARY_KEY_BLOCKS || component == IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS) + if (component == IndexComponentType.PARTITION_SIZES || component == IndexComponentType.PARTITION_KEY_BLOCKS || + component == IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS || component == IndexComponentType.CLUSTERING_KEY_BLOCKS || + component == IndexComponentType.CLUSTERING_KEY_BLOCK_OFFSETS) return; if (((component == IndexComponentType.GROUP_COMPLETION_MARKER) || @@ -1150,7 +1151,7 @@ private void verifyRebuildIndexComponent(IndexContext numericIndexContext, (corruptionType != CorruptionType.REMOVED)) return; - logger.info("CORRUPTING: " + component + ", corruption type = " + corruptionType); + logger.info("CORRUPTING: {}, corruption type = {}", component, corruptionType); int rowCount = 2; diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TokenCollisionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TokenCollisionTest.java index 6debec7c021e..f6304374f63a 100644 --- a/test/unit/org/apache/cassandra/index/sai/cql/TokenCollisionTest.java +++ b/test/unit/org/apache/cassandra/index/sai/cql/TokenCollisionTest.java @@ -24,6 +24,8 @@ import org.junit.Test; import com.datastax.driver.core.Row; +import java.util.ArrayList; + import org.apache.cassandra.Util; import org.apache.cassandra.index.sai.SAITester; import org.apache.cassandra.utils.ByteBufferUtil; @@ -61,4 +63,90 @@ public void testSkippingWhenTokensCollide() // we should match all the rows assertEquals(numRows, rows.size()); } + @Test + public void skinnyPartitionTest() + { + doSkinnyPartitionTest(10, 0); + } + + @Test + public void skinnyPartitionLastRowTest() + { + doSkinnyPartitionTest(49, 9); + } + + private void doSkinnyPartitionTest(int v1Match, int v2Match) + { + createTable("CREATE TABLE %s (pk blob, v1 int, v2 int, PRIMARY KEY (pk))"); + createIndex("CREATE INDEX ON %s(v1) USING 'sai'"); + createIndex("CREATE INDEX ON %s(v2) USING 'sai'"); + + ByteBuffer prefix = ByteBufferUtil.bytes("key"); + int numRows = 100; + int v1Count = 0; + int v2Count = 0; + List matchingPks = new ArrayList<>(); + for (int pkCount = 0; pkCount < numRows; pkCount++) + { + ByteBuffer pk = Util.generateMurmurCollision(prefix, (byte) (pkCount / 64), (byte) (pkCount % 64)); + if (v1Count == v1Match && v2Count == v2Match) + matchingPks.add(row(pk, v1Count, v2Count)); + execute("INSERT INTO %s (pk, v1, v2) VALUES (?, ?, ?)", pk, v1Count++, v2Count++); + if (v1Count == 50) + v1Count = 0; + if (v2Count == 10) + v2Count = 0; + } + assertEquals(2, matchingPks.size()); + flush(); + + assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE v1=" + v1Match + " AND v2=" + v2Match), matchingPks.get(0), matchingPks.get(1)); + } + + @Test + public void widePartitionTest() + { + doWidePartitionTest(100, 10, 0); + } + + @Test + public void widePartitionLastRowTest() + { + // Reduce the number of rows so the last row occurs at the first clustering value + doWidePartitionTest(97, 46, 6); + } + + private void doWidePartitionTest(int numRows, int v1Match, int v2Match) + { + createTable("CREATE TABLE %s (pk blob, ck int, v1 int, v2 int, PRIMARY KEY (pk, ck))"); + createIndex("CREATE INDEX ON %s(v1) USING 'sai'"); + createIndex("CREATE INDEX ON %s(v2) USING 'sai'"); + + ByteBuffer prefix = ByteBufferUtil.bytes("key"); + int pkCount = 0; + int ckCount = 0; + int v1Count = 0; + int v2Count = 0; + List matchingRows = new ArrayList<>(); + for (int i = 0; i < numRows; i++) + { + ByteBuffer pk = Util.generateMurmurCollision(prefix, (byte) (pkCount / 64), (byte) (pkCount % 64)); + if (v1Count == v1Match && v2Count == v2Match) + matchingRows.add(row(pk, ckCount, v1Count, v2Count)); + execute("INSERT INTO %s (pk, ck, v1, v2) VALUES (?, ?, ?, ?)", pk, ckCount++, v1Count++, v2Count++); + if (ckCount == 8) + { + ckCount = 0; + pkCount++; + } + if (v1Count == 50) + v1Count = 0; + if (v2Count == 10) + v2Count = 0; + } + assertEquals(2, matchingRows.size()); + flush(); + + assertRowsIgnoringOrder(execute("SELECT * FROM %s WHERE v1=" + v1Match + " AND v2=" + v2Match), matchingRows.get(0), matchingRows.get(1)); + } } diff --git a/test/unit/org/apache/cassandra/index/sai/disk/NodeStartupTest.java b/test/unit/org/apache/cassandra/index/sai/disk/NodeStartupTest.java index b1e4d005f4d4..aa7bdccf5f65 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/NodeStartupTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/NodeStartupTest.java @@ -176,7 +176,7 @@ enum IndexStateOnRestart PER_SSTABLE_INCOMPLETE, PER_COLUMN_INCOMPLETE, PER_SSTABLE_CORRUPT, - PER_COLUMN_CORRUPT; + PER_COLUMN_CORRUPT } enum StartupTaskRunOrder @@ -349,13 +349,13 @@ public void populateTombstones() } } - private boolean isGroupIndexComplete() throws Exception + private boolean isGroupIndexComplete() { ColumnFamilyStore cfs = Objects.requireNonNull(Schema.instance.getKeyspaceInstance(KEYSPACE)).getColumnFamilyStore(currentTable()); return cfs.getLiveSSTables().stream().allMatch(sstable -> loadDescriptor(sstable, cfs).perSSTableComponents().isComplete()); } - private boolean isColumnIndexComplete() throws Exception + private boolean isColumnIndexComplete() { ColumnFamilyStore cfs = Objects.requireNonNull(Schema.instance.getKeyspaceInstance(KEYSPACE)).getColumnFamilyStore(currentTable()); return cfs.getLiveSSTables().stream().allMatch(sstable -> IndexDescriptor.isIndexBuildCompleteOnDisk(sstable, indexContext)); @@ -368,7 +368,7 @@ private void setState(IndexStateOnRestart state) case VALID: break; case ALL_EMPTY: - Version.current().onDiskFormat().perSSTableComponentTypes().forEach(this::remove); + Version.current().onDiskFormat().perSSTableComponentTypes(false).forEach(this::remove); Version.current().onDiskFormat().perIndexComponentTypes(indexContext).forEach(c -> remove(c, indexContext)); break; case PER_SSTABLE_INCOMPLETE: diff --git a/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java b/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java index ee3d22ec7685..b767c31c4ed0 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/format/IndexDescriptorTest.java @@ -308,7 +308,7 @@ static void createFakePerSSTableComponents(Descriptor descriptor, Version versio static void createFakePerSSTableComponents(Descriptor descriptor, Version version, int generation, int sizeInBytes) throws IOException { - for (IndexComponentType type : version.onDiskFormat().perSSTableComponentTypes()) + for (IndexComponentType type : version.onDiskFormat().perSSTableComponentTypes(false)) createileOnDisk(descriptor, version.fileNameFormatter().format(type, (String)null, generation), sizeInBytes); } diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/LegacyOnDiskFormatTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/LegacyOnDiskFormatTest.java index a21a91effa78..be0badb6c159 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/LegacyOnDiskFormatTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/LegacyOnDiskFormatTest.java @@ -61,6 +61,7 @@ import static org.apache.cassandra.index.sai.disk.v1.kdtree.BKDQueries.bkdQueryFrom; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; /** * Note: The sstables and SAI indexes used in this test were written with DSE 6.8 @@ -158,6 +159,7 @@ public void canCreateAndUsePrimaryKeyMapWithLegacyFormat() throws Throwable assertEquals(countFromFactory, countFromMap); PrimaryKey expected = pkFactory.createTokenOnly(Murmur3Partitioner.instance.decorateKey(Int32Type.instance.decompose(23)).getToken()); + assertTrue(expected.isTokenOnly()); PrimaryKey primaryKey = primaryKeyMap.primaryKeyFromRowId(0); @@ -175,9 +177,9 @@ public void canSearchBDKIndex() throws Throwable List metadatas = SegmentMetadata.load(source, intContext, sstableContext); BKDReader bkdReader = new BKDReader(intContext, - components.get(IndexComponentType.KD_TREE).createFileHandle(), + components.get(IndexComponentType.KD_TREE).createFileHandle(null), metadatas.get(0).getIndexRoot(IndexComponentType.KD_TREE), - components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(), + components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null), metadatas.get(0).getIndexRoot(IndexComponentType.KD_TREE_POSTING_LISTS)); Expression expression = new Expression(intContext).add(Operator.LT, Int32Type.instance.decompose(10)); @@ -203,9 +205,9 @@ public void canSearchTermsIndex() throws Throwable ByteComparable.Version byteComparableVersion = components.byteComparableVersionFor(IndexComponentType.TERMS_DATA); TermsReader termsReader = new TermsReader(textContext, - components.get(IndexComponentType.TERMS_DATA).createFileHandle(), + components.get(IndexComponentType.TERMS_DATA).createFileHandle(null), byteComparableVersion, - components.get(IndexComponentType.POSTING_LISTS).createFileHandle(), + components.get(IndexComponentType.POSTING_LISTS).createFileHandle(null), root, footerPointer, Version.AA); // These tests are for AA, so no need to parameterize diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/SegmentFlushTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/SegmentFlushTest.java index 557f433df267..cc858308e520 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/SegmentFlushTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/SegmentFlushTest.java @@ -202,7 +202,7 @@ private void testFlushBetweenRowIds(long sstableRowId1, long sstableRowId2, int minSegmentRowId = 0; maxSegmentRowId = 0; minKey = SAITester.TEST_FACTORY.createTokenOnly(key2.getToken()); - maxKey = SAITester.TEST_FACTORY.createTokenOnly(key2.getToken());; + maxKey = SAITester.TEST_FACTORY.createTokenOnly(key2.getToken()); minTerm = term2; maxTerm = term2; numRowsPerSegment = 1; @@ -215,8 +215,8 @@ private void testFlushBetweenRowIds(long sstableRowId1, long sstableRowId2, int private void verifyStringIndex(IndexComponents.ForRead components, SegmentMetadata segmentMetadata) throws IOException { - FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); - FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(); + FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); + FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(null); long termsFooterPointer = Long.parseLong(segmentMetadata.componentMetadatas.get(IndexComponentType.TERMS_DATA).attributes.get(SAICodecUtils.FOOTER_POINTER)); diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsReaderTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsReaderTest.java index 764377c3fd00..220095e21808 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsReaderTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsReaderTest.java @@ -117,8 +117,8 @@ private void doTestTermsIteration(Version version) throws IOException indexMetas = writer.writeAll(new MemtableTermsIterator(null, null, iter), docLengths); } - FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); - FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(); + FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); + FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(null); long termsFooterPointer = Long.parseLong(indexMetas.get(IndexComponentType.TERMS_DATA).attributes.get(SAICodecUtils.FOOTER_POINTER)); @@ -160,8 +160,8 @@ private void testTermQueries(Version version, int numTerms, int numPostings) thr indexMetas = writer.writeAll(new MemtableTermsIterator(null, null, iter), docLengths); } - FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); - FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(); + FileHandle termsData = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); + FileHandle postingLists = components.get(IndexComponentType.POSTING_LISTS).createFileHandle(null); long termsFooterPointer = Long.parseLong(indexMetas.get(IndexComponentType.TERMS_DATA).attributes.get(SAICodecUtils.FOOTER_POINTER)); diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bitpack/NumericValuesTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bitpack/NumericValuesTest.java index 0ded82ceff2e..57a8762c80a5 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/bitpack/NumericValuesTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bitpack/NumericValuesTest.java @@ -71,13 +71,13 @@ private void testRepeatedNumericValues(boolean monotonic) throws Exception NumericValuesMeta tokensMeta = new NumericValuesMeta(source.get(tokens)); - try (FileHandle fileHandle = tokens.createFileHandle(); + try (FileHandle fileHandle = tokens.createFileHandle(null); LongArray reader = monotonic ? new MonotonicBlockPackedReader(fileHandle, tokensMeta).open() : new BlockPackedReader(fileHandle, tokensMeta).open()) { for (int x = 0; x < length; x++) { - assertEquals(reader.get(x), 1000); + assertEquals(1000, reader.get(x)); } } } @@ -100,7 +100,7 @@ public void testTokenFind() throws Exception IndexComponent.ForRead tokens = components.get(IndexComponentType.TOKEN_VALUES); NumericValuesMeta tokensMeta = new NumericValuesMeta(source.get(tokens)); - try (FileHandle fileHandle = tokens.createFileHandle(); + try (FileHandle fileHandle = tokens.createFileHandle(null); LongArray reader = new BlockPackedReader(fileHandle, tokensMeta).open()) { assertEquals(array.length, reader.length()); @@ -114,7 +114,7 @@ public void testTokenFind() throws Exception } // non-exact match - try (FileHandle fileHandle = tokens.createFileHandle(); + try (FileHandle fileHandle = tokens.createFileHandle(null); LongArray reader = new BlockPackedReader(fileHandle, tokensMeta).open()) { assertEquals(array.length, reader.length()); @@ -138,7 +138,7 @@ private void testRepeatedNumericValuesFindTokenRowID() throws Exception IndexComponent.ForRead tokens = components.get(IndexComponentType.TOKEN_VALUES); NumericValuesMeta tokensMeta = new NumericValuesMeta(source.get(tokens)); - try (FileHandle fileHandle = tokens.createFileHandle(); + try (FileHandle fileHandle = tokens.createFileHandle(null); LongArray reader = new BlockPackedReader(fileHandle, tokensMeta).open()) { for (int x = 0; x < length; x++) @@ -154,14 +154,14 @@ private void doTest(boolean monotonic) throws Exception { final long[] array = new long[64_000]; final IndexDescriptor indexDescriptor = newIndexDescriptor(); - writeTokens(monotonic, indexDescriptor, array, prev -> monotonic ? prev + nextInt(100) : nextInt(100)); + writeTokens(monotonic, indexDescriptor, array, prev -> monotonic ? prev + nextInt(100) : nextLong(0, Long.MAX_VALUE)); IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); final MetadataSource source = MetadataSource.loadMetadata(components); IndexComponent.ForRead tokens = components.get(IndexComponentType.TOKEN_VALUES); NumericValuesMeta tokensMeta = new NumericValuesMeta(source.get(tokens)); - try (FileHandle fileHandle = tokens.createFileHandle(); + try (FileHandle fileHandle = tokens.createFileHandle(null); LongArray reader = (monotonic ? new MonotonicBlockPackedReader(fileHandle, tokensMeta) : new BlockPackedReader(fileHandle, tokensMeta)).open()) { diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/BKDReaderTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/BKDReaderTest.java index be84f4aadf3a..3b75eb0df294 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/BKDReaderTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/BKDReaderTest.java @@ -412,8 +412,8 @@ private BKDReader finishAndOpenReaderOneDim(int maxPointsPerLeaf, BKDTreeRamBuff final long postingsPosition = metadata.get(IndexComponentType.KD_TREE_POSTING_LISTS).root; MatcherAssert.assertThat(postingsPosition, is(greaterThan(0L))); - FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(); - FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(); + FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(null); + FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null); return new BKDReader(indexContext, kdtreeHandle, bkdPosition, diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/KDTreeIndexBuilder.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/KDTreeIndexBuilder.java index b884227e26a8..ab4bb7405721 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/KDTreeIndexBuilder.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/KDTreeIndexBuilder.java @@ -92,22 +92,22 @@ public PrimaryKey primaryKeyFromRowId(long sstableRowId) } @Override - public long exactRowIdOrInvertedCeiling(PrimaryKey key) + public long rowIdFromPrimaryKey(PrimaryKey key) { return key.token().getLongValue(); } - @Override - public long ceiling(PrimaryKey key) - { - return key.token().getLongValue(); - } - - @Override - public long floor(PrimaryKey key) - { - return key.token().getLongValue(); - } +// @Override +// public long ceiling(PrimaryKey key) +// { +// return key.token().getLongValue(); +// } +// +// @Override +// public long floor(PrimaryKey key) +// { +// return key.token().getLongValue(); +// } @Override public long count() diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/NumericIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/NumericIndexWriterTest.java index 537ec4957e1d..5e861ae82258 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/NumericIndexWriterTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/NumericIndexWriterTest.java @@ -97,8 +97,8 @@ private void doShouldFlushFromRamBuffer() throws Exception indexMetas = writer.writeAll(pointValues); } - final FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(); - final FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(); + final FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(null); + final FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null); try (BKDReader reader = new BKDReader(indexContext, kdtreeHandle, @@ -149,8 +149,8 @@ public void shouldFlushFromMemtable() throws Exception indexMetas = writer.writeAll(pointValues); } - final FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(); - final FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(); + final FileHandle kdtreeHandle = components.get(IndexComponentType.KD_TREE).createFileHandle(null); + final FileHandle kdtreePostingsHandle = components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null); try (BKDReader reader = new BKDReader(indexContext, kdtreeHandle, diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/OneDimBKDPostingsWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/OneDimBKDPostingsWriterTest.java index 7a12ee543a3e..42d44d7fc83f 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/OneDimBKDPostingsWriterTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/kdtree/OneDimBKDPostingsWriterTest.java @@ -81,7 +81,7 @@ public void shouldWritePostingsForEligibleNodes() throws IOException } IndexComponent.ForRead kdTreePostings = components.get(IndexComponentType.KD_TREE_POSTING_LISTS); - BKDPostingsIndex postingsIndex = new BKDPostingsIndex(kdTreePostings.createFileHandle(), fp); + BKDPostingsIndex postingsIndex = new BKDPostingsIndex(kdTreePostings.createFileHandle(null), fp); assertEquals(10, postingsIndex.size()); // Internal postings... @@ -128,7 +128,7 @@ public void shouldSkipPostingListWhenSamplingMisses() throws IOException } // There is only a single posting list...the leaf posting list. - BKDPostingsIndex postingsIndex = new BKDPostingsIndex(components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(), fp); + BKDPostingsIndex postingsIndex = new BKDPostingsIndex(components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null), fp); assertEquals(1, postingsIndex.size()); } @@ -149,7 +149,7 @@ public void shouldSkipPostingListWhenTooFewLeaves() throws IOException } // There is only a single posting list...the leaf posting list. - BKDPostingsIndex postingsIndex = new BKDPostingsIndex(components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(), fp); + BKDPostingsIndex postingsIndex = new BKDPostingsIndex(components.get(IndexComponentType.KD_TREE_POSTING_LISTS).createFileHandle(null), fp); assertEquals(1, postingsIndex.size()); } diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java index ef8ee131b858..4eff795b2962 100644 --- a/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java +++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/trie/TrieTermsDictionaryTest.java @@ -83,7 +83,7 @@ private void doTestExactMatch(Function asByteComparable) fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION)) { assertEquals(NOT_FOUND, reader.exactMatch(asByteComparable.apply("a"))); @@ -170,7 +170,7 @@ private void doTestCeilingStateful(Function asByteCompar fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION)) { assertEquals(0, reader.ceiling(asByteComparable.apply("a"))); @@ -253,7 +253,7 @@ private void doTestCeilingWithEmulatedPrimaryKey(Function asByteComparable) thro fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION)) { assertEquals(NOT_FOUND, reader.floor(asByteComparable.apply("a"))); @@ -322,7 +322,7 @@ private void doTestFloorWithEmulatedPrimaryKey(Function fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION)) { // Validate token only searches @@ -374,7 +374,7 @@ private void doTestTermEnum(Function asByteComparable) t fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader iterator = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION); ReverseTrieTermsDictionaryReader reverseIterator = new ReverseTrieTermsDictionaryReader(input.instantiateRebufferer(), fp)) { @@ -420,7 +420,7 @@ private void doTestMinMaxTerm(Function asByteComparable) fp = writer.complete(new MutableLong()); } - try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(); + try (FileHandle input = components.get(IndexComponentType.TERMS_DATA).createFileHandle(null); TrieTermsDictionaryReader reader = new TrieTermsDictionaryReader(input.instantiateRebufferer(), fp, VERSION)) { final ByteComparable expectedMaxTerm = byteComparables.get(byteComparables.size() - 1); diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v2/WideRowPrimaryKeyTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v2/WideRowPrimaryKeyTest.java new file mode 100644 index 000000000000..5c38edcb0c22 --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/disk/v2/WideRowPrimaryKeyTest.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.disk.v2; + +import java.util.Arrays; + +import com.google.common.base.Stopwatch; +import org.junit.Test; + +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.index.sai.disk.PrimaryKeyMap; +import org.apache.cassandra.index.sai.disk.format.IndexComponents; +import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; +import org.apache.cassandra.index.sai.utils.AbstractPrimaryKeyTest; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.io.sstable.format.SSTableReader; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class WideRowPrimaryKeyTest extends AbstractPrimaryKeyTest +{ + @Test + public void randomTest() throws Throwable + { + IndexDescriptor indexDescriptor = newClusteringIndexDescriptor(compositePartitionMultipleClusteringAsc); + IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); + + SSTableComponentsWriter writer = new SSTableComponentsWriter(components); + + RowAwarePrimaryKeyFactory factory = new RowAwarePrimaryKeyFactory(compositePartitionMultipleClusteringAsc.comparator); + + int rows = nextInt(1000, 10000); + PrimaryKey[] keys = new PrimaryKey[rows]; + int partition = 0; + int partitionSize = nextInt(5, 500); + int partitionCounter = 0; + for (int index = 0; index < rows; index++) + { + keys[index] = factory.create(makeKey(compositePartitionMultipleClusteringAsc, partition, partition), + makeClustering(compositePartitionMultipleClusteringAsc, + randomSimpleString(10, 100), + randomSimpleString(10, 100))); + partitionCounter++; + if (partitionCounter == partitionSize) + { + partition++; + partitionCounter = 0; + partitionSize = nextInt(5, 500); + } + } + + Arrays.sort(keys); + + DecoratedKey lastKey = null; + for (PrimaryKey primaryKey : keys) + { + if (lastKey == null || lastKey.compareTo(primaryKey.partitionKey()) < 0) + { + lastKey = primaryKey.partitionKey(); + writer.startPartition(lastKey); + } + writer.nextRow(primaryKey); + } + + writer.complete(Stopwatch.createStarted()); + + SSTableReader sstableReader = mock(SSTableReader.class); + when(sstableReader.metadata()).thenReturn(compositePartitionMultipleClusteringAsc); + + try (PrimaryKeyMap.Factory mapFactory = new WidePrimaryKeyMap.Factory(components, factory, sstableReader); + PrimaryKeyMap primaryKeyMap = mapFactory.newPerSSTablePrimaryKeyMap()) + { + for (int key = 0; key < rows; key++) + { + PrimaryKey test = keys[key]; + + test = factory.create(test.partitionKey(), + makeClustering(compositePartitionMultipleClusteringAsc, + randomSimpleString(10, 100), + randomSimpleString(10, 100))); + + long rowId = primaryKeyMap.rowIdFromPrimaryKey(test); + + if (rowId >= 0) + { + PrimaryKey found = keys[(int) rowId]; + + assertTrue(found.compareTo(test) >= 0); + + if (rowId > 0) + assertTrue(keys[(int) rowId - 1].compareTo(test) < 0); + } + else + { + assertTrue(test.compareTo(keys[keys.length - 1]) > 0); + } + } + } + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupTest.java new file mode 100644 index 000000000000..63c88cc15a2b --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/disk/v2/keystore/KeyLookupTest.java @@ -0,0 +1,730 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.disk.v2.keystore; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.Before; +import org.junit.Test; + +import org.apache.cassandra.db.Clustering; +import org.apache.cassandra.db.DecoratedKey; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.index.sai.SAITester; +import org.apache.cassandra.index.sai.disk.format.IndexComponentType; +import org.apache.cassandra.index.sai.disk.format.IndexComponents; +import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; +import org.apache.cassandra.index.sai.disk.v1.MetadataSource; +import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; +import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; +import org.apache.cassandra.index.sai.utils.PrimaryKey; +import org.apache.cassandra.index.sai.utils.SAICodecUtils; +import org.apache.cassandra.index.sai.utils.SaiRandomizedTest; +import org.apache.cassandra.index.sai.utils.TypeUtil; +import org.apache.cassandra.io.util.FileHandle; +import org.apache.cassandra.utils.bytecomparable.ByteComparable; +import org.apache.cassandra.utils.bytecomparable.ByteSource; +import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; +import org.apache.lucene.store.IndexInput; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +public class KeyLookupTest extends SaiRandomizedTest +{ + public static final ByteComparable.Version VERSION = TypeUtil.BYTE_COMPARABLE_VERSION; + protected IndexDescriptor indexDescriptor; + + @Before + public void setup() throws Exception + { + indexDescriptor = newIndexDescriptor(); + } + +// @Test +// public void testLexicographicException() throws Exception +// { +// IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); +// try (MetadataWriter metadataWriter = new MetadataWriter(components)) +// { +// NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS), +// metadataWriter, true); +// try (KeyStoreWriter writer = new KeyStoreWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCKS), +// metadataWriter, +// blockFPWriter, +// 4, false)) +// { +// ByteBuffer buffer = Int32Type.instance.decompose(99999); +// ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); +// byte[] bytes1 = ByteSourceInverse.readBytes(byteSource); +// +// writer.add(ByteComparable.preencoded(VERSION, bytes1)); +// +// buffer = Int32Type.instance.decompose(444); +// byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); +// byte[] bytes2 = ByteSourceInverse.readBytes(byteSource); +// +// assertThrows(IllegalArgumentException.class, () -> writer.add(ByteComparable.preencoded(VERSION, bytes2))); +// } +// } +// } + + @Test + public void testFileValidation() throws Exception + { + List primaryKeys = new ArrayList<>(); + + for (int x = 0; x < 11; x++) + { + ByteBuffer buffer = UTF8Type.instance.decompose(Integer.toString(x)); + DecoratedKey partitionKey = Murmur3Partitioner.instance.decorateKey(buffer); + PrimaryKey primaryKey = SAITester.TEST_FACTORY.create(partitionKey, Clustering.EMPTY); + primaryKeys.add(primaryKey); + } + + primaryKeys.sort(PrimaryKey::compareTo); + IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); + + try (MetadataWriter metadataWriter = new MetadataWriter(components)) + { +// IndexOutputWriter bytesWriter = indexDescriptor.openPerSSTableOutput(IndexComponentType.PARTITION_KEY_BLOCKS); + NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS), metadataWriter, true); + try (KeyStoreWriter writer = new KeyStoreWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCKS), + metadataWriter, +// bytesWriter, + blockFPWriter, + 4, + false)) + { + primaryKeys.forEach(primaryKey -> { + try + { + writer.add(primaryKey); + } + catch (IOException e) + { + e.printStackTrace(); + } + }); + } + } + assertTrue(validateComponent(components, IndexComponentType.PARTITION_KEY_BLOCKS, true)); + assertTrue(validateComponent(components, IndexComponentType.PARTITION_KEY_BLOCKS, false)); + assertTrue(validateComponent(components, IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS, true)); + assertTrue(validateComponent(components, IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS, false)); + } + + @Test + public void testSeekToTerm() throws Exception + { + List keys = new ArrayList<>(); + writeTerms(keys); + + // iterate on keys ascending + withKeyLookup(reader -> + { + for (int x = 0; x < keys.size(); x++) + { + try (KeyLookup.Cursor cursor = reader.openCursor()) + { + ByteComparable key = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(VERSION)); + + assertArrayEquals(keys.get(x), bytes); +// long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, keys.get(x))); +// assertEquals(x, pointId); + } + } + }); + + // iterate on keys descending + withKeyLookup(reader -> + { + for (int x = keys.size() - 1; x >= 0; x--) + { + try (KeyLookup.Cursor cursor = reader.openCursor()) + { + ByteComparable key = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(VERSION)); + + assertArrayEquals(keys.get(x), bytes); +// long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, keys.get(x))); +// assertEquals(x, pointId); + } + } + }); + + // iterate randomly + withKeyLookup(reader -> + { + for (int x = 0; x < keys.size(); x++) + { + int target = nextInt(0, keys.size()); + + try (KeyLookup.Cursor cursor = reader.openCursor()) + { + ByteComparable key = cursor.seekToPointId(target); + + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(VERSION)); + + assertArrayEquals(keys.get(target), bytes); +// long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, keys.get(target))); +// assertEquals(target, pointId); + } + } + }); + } + +// @Test +// public void testSeekToTermMinMaxPrefixNoMatch() throws Exception +// { +// IndexDescriptor descriptor = newIndexDescriptor(); +// +// List termsMinPrefixNoMatch = new ArrayList<>(); +// List termsMaxPrefixNoMatch = new ArrayList<>(); +// int valuesPerPrefix = 10; +// writeTerms(descriptor, termsMinPrefixNoMatch, termsMaxPrefixNoMatch, valuesPerPrefix, false); +// +// var countEndOfData = new AtomicInteger(); +// // iterate on terms ascending +// withKeyLookup(reader -> +// { +// for (int x = 0; x < termsMaxPrefixNoMatch.size(); x++) +// { +// try (KeyLookup.Cursor cursor = reader.openCursor()) +// { +// int index = x; +// long pointIdEnd = cursor.ceiling(v -> termsMinPrefixNoMatch.get(index)); +// long pointIdStart = cursor.floor(v -> termsMaxPrefixNoMatch.get(index)); +// if (pointIdStart >= 0 && pointIdEnd >= 0) +// assertTrue(pointIdEnd > pointIdStart); +// else +// countEndOfData.incrementAndGet(); +// } +// } +// }); +// // ceiling reaches the end of the data because we call writeTerms with matchesData false, which means that +// // the last set of terms we are calling ceiling on are greater than anything in the trie, so ceiling returns +// // a negative value. +// assertEquals(valuesPerPrefix, countEndOfData.get()); +// } + +// @Test +// public void testSeekToTermMinMaxPrefix() throws Exception +// { +// IndexDescriptor descriptor = newIndexDescriptor(); +// +// List termsMinPrefix = new ArrayList<>(); +// List termsMaxPrefix = new ArrayList<>(); +// int valuesPerPrefix = 10; +// writeTerms(descriptor, termsMinPrefix, termsMaxPrefix, valuesPerPrefix, true); +// +// // iterate on terms ascending +// withKeyLookup(reader -> +// { +// for (int x = 0; x < termsMaxPrefix.size(); x++) +// { +// try (KeyLookup.Cursor cursor = reader.openCursor()) +// { +// int index = x; +// long pointIdEnd = cursor.ceiling(v -> termsMinPrefix.get(index)); +// long pointIdStart = cursor.floor(v -> termsMaxPrefix.get(index)); +// assertEquals(pointIdEnd, x / valuesPerPrefix * valuesPerPrefix); +// assertEquals(pointIdEnd + valuesPerPrefix - 1, pointIdStart); +// } +// } +// }); +// } + +// @Test +// public void testAdvance() throws Exception +// { +// IndexDescriptor descriptor = newIndexDescriptor(); +// +// List terms = new ArrayList<>(); +// writeTerms(descriptor, terms); +// +// withKeyLookupCursor(cursor -> +// { +// int x = 0; +// while (cursor.advance()) +// { +// ByteComparable term = cursor.term(); +// +// byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); +// assertArrayEquals(terms.get(x), bytes); +// +// x++; +// } +// +// // assert we don't increase the point id beyond one point after the last item +// assertEquals(cursor.pointId(), terms.size()); +// assertFalse(cursor.advance()); +// assertEquals(cursor.pointId(), terms.size()); +// }); +// } + +// @Test +// public void testReset() throws Exception +// { +// IndexDescriptor descriptor = newIndexDescriptor(); +// +// List terms = new ArrayList<>(); +// writeTerms(descriptor, terms); +// +// withKeyLookupCursor(cursor -> +// { +// assertTrue(cursor.advance()); +// assertTrue(cursor.advance()); +// String term1 = cursor.term().byteComparableAsString(VERSION); +// cursor.reset(); +// assertTrue(cursor.advance()); +// assertTrue(cursor.advance()); +// String term2 = cursor.term().byteComparableAsString(VERSION); +// assertEquals(term1, term2); +// assertEquals(1, cursor.pointId()); +// }); +// } + + @Test + public void testLongPrefixesAndSuffixes() throws Exception + { + List keys = new ArrayList<>(); + writeKeys(writer -> { + // The following writes a set of keys that cover the following conditions: + + // Start value 0 + byte[] bytes = new byte[20]; + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // prefix > 15 + bytes = new byte[20]; + Arrays.fill(bytes, 16, 20, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // prefix == 15 + bytes = new byte[20]; + Arrays.fill(bytes, 15, 20, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // prefix < 15 + bytes = new byte[20]; + Arrays.fill(bytes, 14, 20, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // suffix > 16 + bytes = new byte[20]; + Arrays.fill(bytes, 0, 4, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // suffix == 16 + bytes = new byte[20]; + Arrays.fill(bytes, 0, 5, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // suffix < 16 + bytes = new byte[20]; + Arrays.fill(bytes, 0, 6, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + + bytes = new byte[32]; + Arrays.fill(bytes, 0, 16, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + // prefix >= 15 && suffix >= 16 + bytes = new byte[32]; + Arrays.fill(bytes, 0, 32, (byte) 1); + keys.add(bytes); + writer.add(ByteComparable.preencoded(VERSION, bytes)); + }, false); + + doTestKeyLookup(keys); + } + + @Test + public void testNonUniqueKeys() throws Exception + { + List keys = new ArrayList<>(); + + writeKeys(writer -> { + for (int x = 0; x < 4000; x++) + { + ByteBuffer buffer = Int32Type.instance.decompose(5000); + ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, TypeUtil.BYTE_COMPARABLE_VERSION); + byte[] bytes = ByteSourceInverse.readBytes(byteSource); + keys.add(bytes); + + writer.add(ByteComparable.preencoded(VERSION, bytes)); + } + }, false); + + doTestKeyLookup(keys); + } + + @Test + public void testSeekToPointId() throws Exception + { + List keys = new ArrayList<>(); + + writeKeys(writer -> { + for (int x = 0; x < 4000; x++) + { + ByteBuffer buffer = Int32Type.instance.decompose(x); + ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); + byte[] bytes = ByteSourceInverse.readBytes(byteSource); + keys.add(bytes); + + writer.add(ByteComparable.preencoded(VERSION, bytes)); + } + }, false); + + doTestKeyLookup(keys); + } + + @Test + public void testSeekToPointIdCC() throws Exception + { + List terms = new ArrayList<>(); + writeTerms(terms); + + // iterate ascending + withKeyLookupCursor(cursor -> + { + for (int x = 0; x < terms.size(); x++) + { + ByteComparable term = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); + assertArrayEquals(terms.get(x), bytes); + } + }); + + // iterate descending + withKeyLookupCursor(cursor -> + { + for (int x = terms.size() - 1; x >= 0; x--) + { + ByteComparable term = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); + assertArrayEquals(terms.get(x), bytes); + } + }); + + // iterate randomly + withKeyLookupCursor(cursor -> + { + for (int x = 0; x < terms.size(); x++) + { + int target = nextInt(0, terms.size()); + ByteComparable term = cursor.seekToPointId(target); + + byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); + assertArrayEquals(terms.get(target), bytes); + } + }); + } + + @Test + public void testSeekToPointIdOutOfRange() throws Exception + { + writeKeys(writer -> { + for (int x = 0; x < 4000; x++) + { + ByteBuffer buffer = Int32Type.instance.decompose(x); + ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); + byte[] bytes = ByteSourceInverse.readBytes(byteSource); + + writer.add(ByteComparable.preencoded(VERSION, bytes)); + } + }, false); + + withKeyLookupCursor(cursor -> { + assertThatThrownBy(() -> cursor.seekToPointId(-2)).isInstanceOf(IndexOutOfBoundsException.class) + .hasMessage(String.format(KeyLookup.INDEX_OUT_OF_BOUNDS, -2, 4000)); + assertThatThrownBy(() -> cursor.seekToPointId(Long.MAX_VALUE)).isInstanceOf(IndexOutOfBoundsException.class) + .hasMessage(String.format(KeyLookup.INDEX_OUT_OF_BOUNDS, Long.MAX_VALUE, 4000)); + assertThatThrownBy(() -> cursor.seekToPointId(4000)).isInstanceOf(IndexOutOfBoundsException.class) + .hasMessage(String.format(KeyLookup.INDEX_OUT_OF_BOUNDS, 4000, 4000)); + }); + } + + @Test + public void testSeekToKey() throws Exception + { + Map keys = new HashMap<>(); + + writeKeys(writer -> { + long pointId = 0; + for (int x = 0; x < 4000; x += 4) + { + byte[] key = makeKey(x); + keys.put(pointId++, key); + + writer.add(ByteComparable.preencoded(VERSION, key)); + } + }, true); + + withKeyLookupCursor(cursor -> { + assertEquals(0L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(0L)), 0L, 10L)); + cursor.reset(); + assertEquals(160L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(160L)), 160L, 170L)); + cursor.reset(); + assertEquals(165L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(165L)), 160L, 170L)); + cursor.reset(); + assertEquals(175L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(175L)), 160L, 176L)); + cursor.reset(); + assertEquals(176L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(176L)), 160L, 177L)); + cursor.reset(); + assertEquals(176L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(176L)), 175L, 177L)); + cursor.reset(); + assertEquals(176L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, makeKey(701)), 160L, 177L)); + cursor.reset(); + assertEquals(504L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(504L)), 200L, 600L)); + cursor.reset(); + assertEquals(-1L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, makeKey(4000)), 0L, 1000L)); + cursor.reset(); + assertEquals(-1L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, makeKey(4000)), 999L, 1000L)); + cursor.reset(); + assertEquals(999L, cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(999L)), 0L, 1000L)); + }); + } + + @Test + public void seekToKeyOnNonPartitionedTest() throws Throwable + { + Map keys = new HashMap<>(); + + writeKeys(writer -> { + long pointId = 0; + for (int x = 0; x < 16; x += 4) + { + byte[] key = makeKey(x); + keys.put(pointId++, key); + + writer.add(ByteComparable.preencoded(VERSION, key)); + } + }, false); + + withKeyLookupCursor(cursor -> assertThatThrownBy(() -> cursor.clusteredSeekToKey(ByteComparable.preencoded(VERSION, keys.get(0L)), 0L, 10L)) + .isInstanceOf(AssertionError.class)); + } + + @Test + public void partitionedKeysMustBeInOrderInPartitions() throws Throwable + { + writeKeys(writer -> { + writer.startPartition(); + writer.add(ByteComparable.preencoded(VERSION, makeKey(0))); + writer.add(ByteComparable.preencoded(VERSION, makeKey(10))); + assertThatThrownBy(() -> writer.add(ByteComparable.preencoded(VERSION, makeKey(9)))).isInstanceOf(IllegalArgumentException.class); + writer.startPartition(); + writer.add(ByteComparable.preencoded(VERSION, makeKey(9))); + }, true); + } + + private byte[] makeKey(int value) + { + ByteBuffer buffer = Int32Type.instance.decompose(value); + ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, TypeUtil.BYTE_COMPARABLE_VERSION); + return ByteSourceInverse.readBytes(byteSource); + } + + private void doTestKeyLookup(List keys) throws Exception + { + // iterate ascending + withKeyLookupCursor(cursor -> { + for (int x = 0; x < keys.size(); x++) + { + ByteComparable key = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + + assertArrayEquals(keys.get(x), bytes); + } + }); + + // iterate ascending skipping blocks + withKeyLookupCursor(cursor -> { + for (int x = 0; x < keys.size(); x += 17) + { + ByteComparable key = cursor.seekToPointId(x); + + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + + assertArrayEquals(keys.get(x), bytes); + } + }); + + withKeyLookupCursor(cursor -> { + ByteComparable key = cursor.seekToPointId(7); + byte[] bytes = ByteSourceInverse.readBytes(key.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + assertArrayEquals(keys.get(7), bytes); + + key = cursor.seekToPointId(7); + bytes = ByteSourceInverse.readBytes(key.asComparableBytes(TypeUtil.BYTE_COMPARABLE_VERSION)); + assertArrayEquals(keys.get(7), bytes); + }); + } + private void writeTerms(List terms) throws Exception + { +// IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); +// try (MetadataWriter metadataWriter = new MetadataWriter(components)) +// { +// NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), +// metadataWriter, true); +// try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), +// metadataWriter, +// blockFPWriter, +// components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) +// { + writeKeys(writer -> { + + for (int x = 0; x < 1000 * 4; x++) + { + ByteBuffer buffer = Int32Type.instance.decompose(x); + ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); + byte[] bytes = ByteSourceInverse.readBytes(byteSource); + terms.add(bytes); + + writer.add(ByteComparable.preencoded(VERSION, bytes)); + } + }, false); +// } +// components.markComplete(); + } + + private void writeTerms(List termsMinPrefix, List termsMaxPrefix, int numPerPrefix, boolean matchesData) throws Exception + { +// IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); +// try (MetadataWriter metadataWriter = new MetadataWriter(components)) +// { +// NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), +// metadataWriter, true); +// try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), +// metadataWriter, +// blockFPWriter, +// components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) +// { + writeKeys(writer -> { + for (int x = 0; x < 1000 ; x++) + { + int component1 = x * 2; + for (int i = 0; i < numPerPrefix; i++) + { + String component2 = "v" + i; + termsMinPrefix.add(ByteSource.withTerminator(ByteSource.LT_NEXT_COMPONENT, intByteSource(component1 + (matchesData ? 0 : 1)))); + termsMaxPrefix.add(ByteSource.withTerminator(ByteSource.GT_NEXT_COMPONENT, intByteSource(component1 + (matchesData ? 0 : 1)))); + writer.add(v -> ByteSource.withTerminator(ByteSource.TERMINATOR, intByteSource(component1), utfByteSource(component2))); + } + } + }, false); +// } +// components.markComplete(); + } + + private ByteSource intByteSource(int value) + { + ByteBuffer buffer = Int32Type.instance.decompose(value); + return Int32Type.instance.asComparableBytes(buffer, VERSION); + } + + private ByteSource utfByteSource(String value) + { + ByteBuffer buffer = UTF8Type.instance.decompose(value); + return UTF8Type.instance.asComparableBytes(buffer, VERSION); + } + + protected void writeKeys(ThrowingConsumer testCode, boolean clustering) throws Exception + { + IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); + try (MetadataWriter metadataWriter = new MetadataWriter(components)) + { +// IndexOutputWriter bytesWriter = indexDescriptor.openPerSSTableOutput(IndexComponent.PARTITION_KEY_BLOCKS); + NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS), metadataWriter, true); + try (KeyStoreWriter writer = new KeyStoreWriter(components.addOrGet(IndexComponentType.PARTITION_KEY_BLOCKS), + metadataWriter, +// bytesWriter, + blockFPWriter, + 4, + clustering)) + { + testCode.accept(writer); + } + } + components.markComplete(); + } + + @FunctionalInterface + public interface ThrowingConsumer + { + void accept(T t) throws Exception; + } + + private void withKeyLookup(ThrowingConsumer testCode) throws Exception + { + IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); + MetadataSource metadataSource = MetadataSource.loadMetadata(components); + NumericValuesMeta blockPointersMeta = new NumericValuesMeta(metadataSource.get(components.get(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS))); + KeyLookupMeta keyLookupMeta = new KeyLookupMeta(metadataSource.get(components.get(IndexComponentType.PARTITION_KEY_BLOCKS))); + try (FileHandle keysData = components.get(IndexComponentType.PARTITION_KEY_BLOCKS).createFileHandle(null); + FileHandle blockOffsets = components.get(IndexComponentType.PARTITION_KEY_BLOCK_OFFSETS).createFileHandle(null)) + { + KeyLookup reader = new KeyLookup(keysData, blockOffsets, keyLookupMeta, blockPointersMeta); + testCode.accept(reader); + } + } + + private void withKeyLookupCursor(ThrowingConsumer testCode) throws Exception + { + withKeyLookup(reader -> { + try (KeyLookup.Cursor cursor = reader.openCursor()) + { + testCode.accept(cursor); + } + }); + } + + private boolean validateComponent(IndexComponents.ForRead components, IndexComponentType indexComponentType, boolean checksum) + { + try (IndexInput input = components.get(indexComponentType).openInput()) + { + if (checksum) + SAICodecUtils.validateChecksum(input); + else + SAICodecUtils.validate(input); + return true; + } + catch (Throwable ignored) + { + } + return false; + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsTest.java deleted file mode 100644 index b2a12ad0c75b..000000000000 --- a/test/unit/org/apache/cassandra/index/sai/disk/v2/sortedterms/SortedTermsTest.java +++ /dev/null @@ -1,484 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.cassandra.index.sai.disk.v2.sortedterms; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; - -import org.junit.Test; - -import org.apache.cassandra.db.Clustering; -import org.apache.cassandra.db.DecoratedKey; -import org.apache.cassandra.db.marshal.Int32Type; -import org.apache.cassandra.db.marshal.UTF8Type; -import org.apache.cassandra.dht.Murmur3Partitioner; -import org.apache.cassandra.index.sai.SAITester; -import org.apache.cassandra.index.sai.disk.format.IndexComponents; -import org.apache.cassandra.index.sai.disk.format.IndexComponentType; -import org.apache.cassandra.index.sai.disk.format.IndexComponent; -import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; -import org.apache.cassandra.index.sai.disk.v1.MetadataSource; -import org.apache.cassandra.index.sai.disk.v1.MetadataWriter; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesMeta; -import org.apache.cassandra.index.sai.disk.v1.bitpack.NumericValuesWriter; -import org.apache.cassandra.index.sai.utils.PrimaryKey; -import org.apache.cassandra.index.sai.utils.SAICodecUtils; -import org.apache.cassandra.index.sai.utils.SaiRandomizedTest; -import org.apache.cassandra.index.sai.utils.TypeUtil; -import org.apache.cassandra.io.util.FileHandle; -import org.apache.cassandra.utils.bytecomparable.ByteComparable; -import org.apache.cassandra.utils.bytecomparable.ByteSource; -import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse; -import org.apache.lucene.store.IndexInput; - -public class SortedTermsTest extends SaiRandomizedTest -{ - - public static final ByteComparable.Version VERSION = TypeUtil.BYTE_COMPARABLE_VERSION; - - @Test - public void testLexicographicException() throws Exception - { - IndexDescriptor indexDescriptor = newIndexDescriptor(); - IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); - try (MetadataWriter metadataWriter = new MetadataWriter(components)) - { - NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) - { - ByteBuffer buffer = Int32Type.instance.decompose(99999); - ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); - byte[] bytes1 = ByteSourceInverse.readBytes(byteSource); - - writer.add(ByteComparable.preencoded(VERSION, bytes1)); - - buffer = Int32Type.instance.decompose(444); - byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); - byte[] bytes2 = ByteSourceInverse.readBytes(byteSource); - - assertThrows(IllegalArgumentException.class, () -> writer.add(ByteComparable.preencoded(VERSION, bytes2))); - } - } - } - - @Test - public void testFileValidation() throws Exception - { - IndexDescriptor indexDescriptor = newIndexDescriptor(); - - List primaryKeys = new ArrayList<>(); - - for (int x = 0; x < 11; x++) - { - ByteBuffer buffer = UTF8Type.instance.decompose(Integer.toString(x)); - DecoratedKey partitionKey = Murmur3Partitioner.instance.decorateKey(buffer); - PrimaryKey primaryKey = SAITester.TEST_FACTORY.create(partitionKey, Clustering.EMPTY); - primaryKeys.add(primaryKey); - } - - primaryKeys.sort(PrimaryKey::compareTo); - - IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); - try (MetadataWriter metadataWriter = new MetadataWriter(components)) - { - NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) - { - primaryKeys.forEach(primaryKey -> { - try - { - writer.add(v -> primaryKey.asComparableBytes(v)); - } - catch (IOException e) - { - e.printStackTrace(); - } - }); - } - } - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_TRIE, true)); - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_TRIE, false)); - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_BLOCKS, true)); - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_BLOCKS, false)); - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS, true)); - assertTrue(validateComponent(components, IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS, false)); - } - - @Test - public void testSeekToTerm() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List terms = new ArrayList<>(); - writeTerms(descriptor, terms); - - // iterate on terms ascending - withSortedTermsReader(descriptor, reader -> - { - for (int x = 0; x < terms.size(); x++) - { - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, terms.get(x))); - assertEquals(x, pointId); - } - } - }); - - // iterate on terms descending - withSortedTermsReader(descriptor, reader -> - { - for (int x = terms.size() - 1; x >= 0; x--) - { - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, terms.get(x))); - assertEquals(x, pointId); - } - } - }); - - // iterate randomly - withSortedTermsReader(descriptor, reader -> - { - for (int x = 0; x < terms.size(); x++) - { - int target = nextInt(0, terms.size()); - - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - long pointId = cursor.ceiling(ByteComparable.preencoded(VERSION, terms.get(target))); - assertEquals(target, pointId); - } - } - }); - } - - @Test - public void testSeekToTermMinMaxPrefixNoMatch() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List termsMinPrefixNoMatch = new ArrayList<>(); - List termsMaxPrefixNoMatch = new ArrayList<>(); - int valuesPerPrefix = 10; - writeTerms(descriptor, termsMinPrefixNoMatch, termsMaxPrefixNoMatch, valuesPerPrefix, false); - - var countEndOfData = new AtomicInteger(); - // iterate on terms ascending - withSortedTermsReader(descriptor, reader -> - { - for (int x = 0; x < termsMaxPrefixNoMatch.size(); x++) - { - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - int index = x; - long pointIdEnd = cursor.ceiling(v -> termsMinPrefixNoMatch.get(index)); - long pointIdStart = cursor.floor(v -> termsMaxPrefixNoMatch.get(index)); - if (pointIdStart >= 0 && pointIdEnd >= 0) - assertTrue(pointIdEnd > pointIdStart); - else - countEndOfData.incrementAndGet(); - } - } - }); - // ceiling reaches the end of the data because we call writeTerms with matchesData false, which means that - // the last set of terms we are calling ceiling on are greater than anything in the trie, so ceiling returns - // a negative value. - assertEquals(valuesPerPrefix, countEndOfData.get()); - } - - @Test - public void testSeekToTermMinMaxPrefix() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List termsMinPrefix = new ArrayList<>(); - List termsMaxPrefix = new ArrayList<>(); - int valuesPerPrefix = 10; - writeTerms(descriptor, termsMinPrefix, termsMaxPrefix, valuesPerPrefix, true); - - // iterate on terms ascending - withSortedTermsReader(descriptor, reader -> - { - for (int x = 0; x < termsMaxPrefix.size(); x++) - { - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - int index = x; - long pointIdEnd = cursor.ceiling(v -> termsMinPrefix.get(index)); - long pointIdStart = cursor.floor(v -> termsMaxPrefix.get(index)); - assertEquals(pointIdEnd, x / valuesPerPrefix * valuesPerPrefix); - assertEquals(pointIdEnd + valuesPerPrefix - 1, pointIdStart); - } - } - }); - } - - @Test - public void testAdvance() throws IOException - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List terms = new ArrayList<>(); - writeTerms(descriptor, terms); - - withSortedTermsCursor(descriptor, cursor -> - { - int x = 0; - while (cursor.advance()) - { - ByteComparable term = cursor.term(); - - byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); - assertArrayEquals(terms.get(x), bytes); - - x++; - } - - // assert we don't increase the point id beyond one point after the last item - assertEquals(cursor.pointId(), terms.size()); - assertFalse(cursor.advance()); - assertEquals(cursor.pointId(), terms.size()); - }); - } - - @Test - public void testReset() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List terms = new ArrayList<>(); - writeTerms(descriptor, terms); - - withSortedTermsCursor(descriptor, cursor -> - { - assertTrue(cursor.advance()); - assertTrue(cursor.advance()); - String term1 = cursor.term().byteComparableAsString(VERSION); - cursor.reset(); - assertTrue(cursor.advance()); - assertTrue(cursor.advance()); - String term2 = cursor.term().byteComparableAsString(VERSION); - assertEquals(term1, term2); - assertEquals(1, cursor.pointId()); - }); - } - - @Test - public void testSeekToPointId() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List terms = new ArrayList<>(); - writeTerms(descriptor, terms); - - // iterate ascending - withSortedTermsCursor(descriptor, cursor -> - { - for (int x = 0; x < terms.size(); x++) - { - cursor.seekToPointId(x); - ByteComparable term = cursor.term(); - - byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); - assertArrayEquals(terms.get(x), bytes); - } - }); - - // iterate descending - withSortedTermsCursor(descriptor, cursor -> - { - for (int x = terms.size() - 1; x >= 0; x--) - { - cursor.seekToPointId(x); - ByteComparable term = cursor.term(); - - byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); - assertArrayEquals(terms.get(x), bytes); - } - }); - - // iterate randomly - withSortedTermsCursor(descriptor, cursor -> - { - for (int x = 0; x < terms.size(); x++) - { - int target = nextInt(0, terms.size()); - cursor.seekToPointId(target); - ByteComparable term = cursor.term(); - - byte[] bytes = ByteSourceInverse.readBytes(term.asComparableBytes(VERSION)); - assertArrayEquals(terms.get(target), bytes); - } - }); - } - - @Test - public void testSeekToPointIdOutOfRange() throws Exception - { - IndexDescriptor descriptor = newIndexDescriptor(); - - List terms = new ArrayList<>(); - writeTerms(descriptor, terms); - - withSortedTermsCursor(descriptor, cursor -> - { - assertThrows(IndexOutOfBoundsException.class, () -> cursor.seekToPointId(-2)); - assertThrows(IndexOutOfBoundsException.class, () -> cursor.seekToPointId(Long.MAX_VALUE)); - }); - } - - private void writeTerms(IndexDescriptor indexDescriptor, List terms) throws IOException - { - IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); - try (MetadataWriter metadataWriter = new MetadataWriter(components)) - { - NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) - { - for (int x = 0; x < 1000 * 4; x++) - { - ByteBuffer buffer = Int32Type.instance.decompose(x); - ByteSource byteSource = Int32Type.instance.asComparableBytes(buffer, VERSION); - byte[] bytes = ByteSourceInverse.readBytes(byteSource); - terms.add(bytes); - - writer.add(ByteComparable.preencoded(VERSION, bytes)); - } - } - } - components.markComplete(); - } - - private void writeTerms(IndexDescriptor indexDescriptor, List termsMinPrefix, List termsMaxPrefix, int numPerPrefix, boolean matchesData) throws IOException - { - IndexComponents.ForWrite components = indexDescriptor.newPerSSTableComponentsForWrite(); - try (MetadataWriter metadataWriter = new MetadataWriter(components)) - { - NumericValuesWriter blockFPWriter = new NumericValuesWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS), - metadataWriter, true); - try (SortedTermsWriter writer = new SortedTermsWriter(components.addOrGet(IndexComponentType.PRIMARY_KEY_BLOCKS), - metadataWriter, - blockFPWriter, - components.addOrGet(IndexComponentType.PRIMARY_KEY_TRIE))) - { - for (int x = 0; x < 1000 ; x++) - { - int component1 = x * 2; - for (int i = 0; i < numPerPrefix; i++) - { - String component2 = "v" + i; - termsMinPrefix.add(ByteSource.withTerminator(ByteSource.LT_NEXT_COMPONENT, intByteSource(component1 + (matchesData ? 0 : 1)))); - termsMaxPrefix.add(ByteSource.withTerminator(ByteSource.GT_NEXT_COMPONENT, intByteSource(component1 + (matchesData ? 0 : 1)))); - writer.add(v -> ByteSource.withTerminator(ByteSource.TERMINATOR, intByteSource(component1), utfByteSource(component2))); - } - } - } - } - components.markComplete(); - } - - private ByteSource intByteSource(int value) - { - ByteBuffer buffer = Int32Type.instance.decompose(value); - return Int32Type.instance.asComparableBytes(buffer, VERSION); - } - - private ByteSource utfByteSource(String value) - { - ByteBuffer buffer = UTF8Type.instance.decompose(value); - return UTF8Type.instance.asComparableBytes(buffer, VERSION); - } - - @FunctionalInterface - public interface ThrowingConsumer { - void accept(T t) throws IOException; - } - - private void withSortedTermsCursor(IndexDescriptor indexDescriptor, - ThrowingConsumer testCode) throws IOException - { - IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); - MetadataSource metadataSource = MetadataSource.loadMetadata(components); - IndexComponent.ForRead blocksComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCKS); - IndexComponent.ForRead blockOffsetsComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS); - NumericValuesMeta blockPointersMeta = new NumericValuesMeta(metadataSource.get(blockOffsetsComponent.fileNamePart())); - SortedTermsMeta sortedTermsMeta = new SortedTermsMeta(metadataSource.get(blocksComponent.fileNamePart())); - try (FileHandle trieHandle = components.get(IndexComponentType.PRIMARY_KEY_TRIE).createFileHandle(); - FileHandle termsData = blocksComponent.createFileHandle(); - FileHandle blockOffsets = blockOffsetsComponent.createFileHandle()) - { - SortedTermsReader reader = new SortedTermsReader(termsData, blockOffsets, trieHandle, sortedTermsMeta, blockPointersMeta); - try (SortedTermsReader.Cursor cursor = reader.openCursor()) - { - testCode.accept(cursor); - } - } - } - - private void withSortedTermsReader(IndexDescriptor indexDescriptor, - ThrowingConsumer testCode) throws IOException - { - IndexComponents.ForRead components = indexDescriptor.perSSTableComponents(); - MetadataSource metadataSource = MetadataSource.loadMetadata(components); - IndexComponent.ForRead blocksComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCKS); - IndexComponent.ForRead blockOffsetsComponent = components.get(IndexComponentType.PRIMARY_KEY_BLOCK_OFFSETS); - NumericValuesMeta blockPointersMeta = new NumericValuesMeta(metadataSource.get(blockOffsetsComponent.fileNamePart())); - SortedTermsMeta sortedTermsMeta = new SortedTermsMeta(metadataSource.get(blocksComponent.fileNamePart())); - try (FileHandle trieHandle = components.get(IndexComponentType.PRIMARY_KEY_TRIE).createFileHandle(); - FileHandle termsData = blocksComponent.createFileHandle(); - FileHandle blockOffsets = blockOffsetsComponent.createFileHandle()) - { - SortedTermsReader reader = new SortedTermsReader(termsData, blockOffsets, trieHandle, sortedTermsMeta, blockPointersMeta); - testCode.accept(reader); - } - } - - private boolean validateComponent(IndexComponents.ForRead components, IndexComponentType indexComponentType, boolean checksum) - { - try (IndexInput input = components.get(indexComponentType).openInput()) - { - if (checksum) - SAICodecUtils.validateChecksum(input); - else - SAICodecUtils.validate(input); - return true; - } - catch (Throwable e) - { - } - return false; - } -} diff --git a/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java b/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java index 9fb719b3dd5e..65d9b841ee02 100644 --- a/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java +++ b/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java @@ -52,7 +52,7 @@ public void testInvalidateWithoutObsolete() SSTableReader sstable = Iterables.getOnlyElement(cfs.getLiveSSTables()); Set components = group.activeComponents(sstable); - assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes().size() + 1, components.size()); + assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes(false).size() + 1, components.size()); // index files are released but not removed cfs.invalidate(true, false); @@ -77,7 +77,7 @@ public void getLiveComponentsForEmptyIndex() Set components = group.activeComponents(sstables.iterator().next()); - assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes().size() + 1, components.size()); + assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes(false).size() + 1, components.size()); } @Test @@ -96,7 +96,7 @@ public void getLiveComponentsForPopulatedIndex() Set components = group.activeComponents(sstables.iterator().next()); - assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes().size() + + assertEquals(Version.current().onDiskFormat().perSSTableComponentTypes(false).size() + Version.current().onDiskFormat().perIndexComponentTypes(indexContext).size(), components.size()); } diff --git a/test/unit/org/apache/cassandra/index/sai/functional/IndexBuildDeciderTest.java b/test/unit/org/apache/cassandra/index/sai/functional/IndexBuildDeciderTest.java index b10b78060730..6e7c96664b54 100644 --- a/test/unit/org/apache/cassandra/index/sai/functional/IndexBuildDeciderTest.java +++ b/test/unit/org/apache/cassandra/index/sai/functional/IndexBuildDeciderTest.java @@ -151,7 +151,7 @@ private int numericIndexFileCount() { IndexContext context = createIndexContext("v1", Int32Type.instance); return V2OnDiskFormat.instance.perIndexComponentTypes(context).size() - + V2OnDiskFormat.instance.perSSTableComponentTypes().size(); + + V2OnDiskFormat.instance.perSSTableComponentTypes(false).size(); } public static class IndexBuildDeciderWithoutInitialBuild implements IndexBuildDecider diff --git a/test/unit/org/apache/cassandra/index/sai/functional/SaiDiskSizeTest.java b/test/unit/org/apache/cassandra/index/sai/functional/SaiDiskSizeTest.java new file mode 100644 index 000000000000..4a12bb5fd508 --- /dev/null +++ b/test/unit/org/apache/cassandra/index/sai/functional/SaiDiskSizeTest.java @@ -0,0 +1,196 @@ +/* + * Copyright DataStax, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.index.sai.functional; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Date; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.junit.Before; +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.index.sai.SAITester; +import org.apache.cassandra.index.sai.SAIUtil; +import org.apache.cassandra.index.sai.disk.format.Version; +import org.apache.cassandra.locator.InetAddressAndPort; + +import static org.assertj.core.api.Assertions.assertThat; + +@RunWith(Parameterized.class) +public class SaiDiskSizeTest extends SAITester +{ + private static final Logger logger = LoggerFactory.getLogger(SaiDiskSizeTest.class); + + @Parameterized.Parameter + public Version version; + + @Parameterized.Parameter(1) + public int size; + + @Parameterized.Parameter(2) + public String pkSuffix; + + @Parameterized.Parameter(3) + public int rowsPerPartition; + + @Parameterized.Parameters(name = "{0}, size={1}, pk={2}, partitionSize={3}") + public static Collection data() + { + return Version.ALL.stream() + .flatMap(v -> { + switch (v.toString()) + { + case "aa": + return Stream.of( + new Object[]{ v, 13014, "pk", 1 }, + new Object[]{ v, 13016, "pk, v_int", 2 }, + new Object[]{ v, 14266, "pk, v_int", 100 }); + case "ba": + case "ca": + case "db": + case "dc": + return Stream.of( + new Object[]{ v, 65734, "pk", 1 }, + new Object[]{ v, 57627, "pk, v_int", 2 }, + new Object[]{ v, 28004, "pk, v_int", 100 }); + case "eb": + case "ec": + return Stream.of( + new Object[]{ v, 67370, "pk", 1 }, + new Object[]{ v, 59263, "pk, v_int", 2 }, + new Object[]{ v, 29640, "pk, v_int", 100 }); + case "ed": + return Stream.of( + new Object[]{ v, 67378, "pk", 1 }, + new Object[]{ v, 59271, "pk, v_int", 2 }, + new Object[]{ v, 29648, "pk, v_int", 100 }); + default: + return // A new version assumes the latest size by default + Stream.of( + new Object[]{ v, 67378, "pk", 1 }, + new Object[]{ v, 59271, "pk, v_int", 2 }, + new Object[]{ v, 29648, "pk, v_int", 100 }); + } + }) + .collect(Collectors.toList()); + } + + @Before + public void setVersion() + { + SAIUtil.setCurrentVersion(version); + } + + @Test + public void testIndexDiskSizeAcrossVersions() throws UnknownHostException + { + createTable("CREATE TABLE %s (" + + "pk int, " + + "v_ascii ascii, " + + "v_bigint bigint, " + + "v_blob blob, " + + "v_boolean boolean, " + + "v_decimal decimal, " + + "v_double double, " + + "v_float float, " + + "v_int int, " + + "v_text text, " + + "v_timestamp timestamp, " + + "v_uuid uuid, " + + "v_varchar varchar, " + + "v_varint varint, " + + "v_timeuuid timeuuid, " + + "v_inet inet, " + + "v_date date, " + + "v_time time, " + + "v_smallint smallint, " + + "v_tinyint tinyint, " + + "v_duration duration, " + + "PRIMARY KEY (" + pkSuffix + "))"); + + verifyNoIndexFiles(); + createIndex("CREATE CUSTOM INDEX ON %s(v_int) USING 'StorageAttachedIndex'"); + + waitForTableIndexesQueryable(); + + // Split data into 2 sstable segments + insertRows(0, 1000); + flush(); + insertRows(1000, 1000); + flush(); + + long diskSize = indexDiskSpaceUse(); + logger.info("SAI Version: {}, Index Disk Size: {} bytes", version, diskSize); + assertThat(diskSize) + .as("Disk size for SAI version %s", version) + .isLessThanOrEqualTo(size) + .isGreaterThan((long) (size * 0.8)); + + compact(); + + diskSize = indexDiskSpaceUse(); + logger.info("SAI Version: {}, Index Disk Size: {} bytes", version, diskSize); + assertThat(diskSize) + .as("Disk size for SAI version %s", version) + .isLessThanOrEqualTo(size) + .isGreaterThan((long) (size * 0.8)); + } + + private void insertRows(int size, int start) throws UnknownHostException + { + for (int i = start; i < start + size; i++) + { + execute("INSERT INTO %s (pk, v_ascii, v_bigint, v_blob, v_boolean, v_decimal, " + + "v_double, v_float, v_int, v_text, v_timestamp, v_uuid, v_varchar, " + + "v_varint, v_timeuuid, v_inet, v_date, v_time, v_smallint, v_tinyint, v_duration) " + + "VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)", + i % (size / rowsPerPartition), // Have 2 rows per partition + "ascii_" + i, + (long) i * 1000000, + ByteBuffer.wrap(("blob_" + i).getBytes()), + i % 2 == 0, + new BigDecimal(i + ".123"), + i * 1.5, + (float) (i * 2.5), + i * 2, + "text_value_" + i, + new Date(System.currentTimeMillis() + i * 1000L), + UUID.randomUUID(), + "varchar_" + i, + BigInteger.valueOf(i).multiply(BigInteger.valueOf(100)), + UUID.fromString("00000000-0000-1000-8000-" + + String.format("%012d", i)), + InetAddressAndPort.getByName("127.0.0." + (i % 256)).address, + i + 1, + (long) i * 1000000000L, + (short) (i % 32767), + (byte) (i % 128), + org.apache.cassandra.cql3.Duration.newInstance(i % 12, i % 30, i * 1000000000L) + ); + } + } +} diff --git a/test/unit/org/apache/cassandra/index/sai/iterators/AbstractKeyRangeIteratorTest.java b/test/unit/org/apache/cassandra/index/sai/iterators/AbstractKeyRangeIteratorTest.java index cd778ad839e9..6b87099f0441 100644 --- a/test/unit/org/apache/cassandra/index/sai/iterators/AbstractKeyRangeIteratorTest.java +++ b/test/unit/org/apache/cassandra/index/sai/iterators/AbstractKeyRangeIteratorTest.java @@ -182,8 +182,8 @@ static Pair createRandomIterator() } - private static final PrimaryKey.Factory TEST_PRIMARY_KEY_FACTORY = Version.current().onDiskFormat() - .newPrimaryKeyFactory(new ClusteringComparator(LongType.instance)); + protected static final PrimaryKey.Factory TEST_PRIMARY_KEY_FACTORY = Version.current().onDiskFormat() + .newPrimaryKeyFactory(new ClusteringComparator(LongType.instance)); /** * Generates a random list of primary keys with the given average number of partitions and rows per partition. diff --git a/test/unit/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIteratorTest.java b/test/unit/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIteratorTest.java index 033fa02d718a..7410ed7ef4e4 100644 --- a/test/unit/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIteratorTest.java +++ b/test/unit/org/apache/cassandra/index/sai/iterators/KeyRangeConcatIteratorTest.java @@ -23,6 +23,7 @@ import org.junit.Test; +import org.apache.cassandra.dht.Murmur3Partitioner; import org.apache.cassandra.utils.Pair; import static org.apache.cassandra.index.sai.iterators.LongIterator.convert; @@ -439,4 +440,11 @@ private KeyRangeIterator.Builder getConcatBuilder() { return KeyRangeConcatIterator.builder(); } + + private String createErrorMessage(int max, int min) + { + return String.format(KeyRangeConcatIterator.MUST_BE_SORTED_ERROR, + TEST_PRIMARY_KEY_FACTORY.createTokenOnly(new Murmur3Partitioner.LongToken(max)), + TEST_PRIMARY_KEY_FACTORY.createTokenOnly(new Murmur3Partitioner.LongToken(min))); + } } diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java b/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java index b514db52058c..062d7a14271a 100644 --- a/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java +++ b/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java @@ -66,7 +66,7 @@ public void verifyIndexGroupMetrics() throws Throwable // with 10 sstable int indexopenFileCountWithOnlyNumeric = getOpenIndexFiles(); - assertEquals(sstables * (Version.current().onDiskFormat().openFilesPerSSTable() + + assertEquals(sstables * (Version.current().onDiskFormat().openFilesPerSSTable(false) + Version.current().onDiskFormat().openFilesPerIndex(v1IndexContext)), indexopenFileCountWithOnlyNumeric); @@ -89,14 +89,14 @@ public void verifyIndexGroupMetrics() throws Throwable compact(); long perSSTableFileDiskUsage = getDiskUsage(); - assertEquals(Version.current().onDiskFormat().openFilesPerSSTable() + + assertEquals(Version.current().onDiskFormat().openFilesPerSSTable(false) + Version.current().onDiskFormat().openFilesPerIndex(v2IndexContext) + Version.current().onDiskFormat().openFilesPerIndex(v1IndexContext), getOpenIndexFiles()); // drop string index, reduce open string index files, per-sstable file disk usage remains the same dropIndex("DROP INDEX %s." + v2IndexName); - assertEquals(Version.current().onDiskFormat().openFilesPerSSTable() + + assertEquals(Version.current().onDiskFormat().openFilesPerSSTable(false) + Version.current().onDiskFormat().openFilesPerIndex(v1IndexContext), getOpenIndexFiles()); assertEquals(perSSTableFileDiskUsage, getDiskUsage()); diff --git a/test/unit/org/apache/cassandra/index/sai/utils/AbstractPrimaryKeyTest.java b/test/unit/org/apache/cassandra/index/sai/utils/AbstractPrimaryKeyTest.java index 1353fc6fbb76..a228c8eccc7b 100644 --- a/test/unit/org/apache/cassandra/index/sai/utils/AbstractPrimaryKeyTest.java +++ b/test/unit/org/apache/cassandra/index/sai/utils/AbstractPrimaryKeyTest.java @@ -76,13 +76,13 @@ public class AbstractPrimaryKeyTest extends SaiRandomizedTest .addClusteringColumn("ck1", UTF8Type.instance) .build(); - static TableMetadata compositePartitionMultipleClusteringAsc = TableMetadata.builder("test", "test") - .partitioner(Murmur3Partitioner.instance) - .addPartitionKeyColumn("pk1", Int32Type.instance) - .addPartitionKeyColumn("pk2", Int32Type.instance) - .addClusteringColumn("ck1", UTF8Type.instance) - .addClusteringColumn("ck2", UTF8Type.instance) - .build(); + protected static TableMetadata compositePartitionMultipleClusteringAsc = TableMetadata.builder("test", "test") + .partitioner(Murmur3Partitioner.instance) + .addPartitionKeyColumn("pk1", Int32Type.instance) + .addPartitionKeyColumn("pk2", Int32Type.instance) + .addClusteringColumn("ck1", UTF8Type.instance) + .addClusteringColumn("ck2", UTF8Type.instance) + .build(); static TableMetadata compositePartitionSingleClusteringDesc = TableMetadata.builder("test", "test") .partitioner(Murmur3Partitioner.instance) @@ -116,12 +116,12 @@ public class AbstractPrimaryKeyTest extends SaiRandomizedTest void assertByteComparison(PrimaryKey a, PrimaryKey b, int expected) { - assertEquals(expected, ByteComparable.compare(v -> a.asComparableBytes(v), - v -> b.asComparableBytes(v), + assertEquals(expected, ByteComparable.compare(a::asComparableBytes, + b::asComparableBytes, TypeUtil.BYTE_COMPARABLE_VERSION)); } - void assertCompareToAndEquals(PrimaryKey a, PrimaryKey b, int expected) + protected void assertCompareToAndEquals(PrimaryKey a, PrimaryKey b, int expected) { if (expected > 0) { @@ -135,22 +135,22 @@ else if (expected < 0) } else { - assertTrue(a.compareTo(b) == 0); + assertEquals(0, a.compareTo(b)); assertEquals(a, b); } } - DecoratedKey makeKey(TableMetadata table, Object...partitionKeys) + protected DecoratedKey makeKey(TableMetadata table, Object... partitionKeys) { ByteBuffer key; if (TypeUtil.isComposite(table.partitionKeyType)) key = ((CompositeType)table.partitionKeyType).decompose(partitionKeys); else - key = table.partitionKeyType.fromString((String)partitionKeys[0]); + key = table.partitionKeyType.decomposeUntyped(partitionKeys[0]); return table.partitioner.decorateKey(key); } - Clustering makeClustering(TableMetadata table, String...clusteringKeys) + protected Clustering makeClustering(TableMetadata table, String... clusteringKeys) { Clustering clustering; if (table.comparator.size() == 0) @@ -159,7 +159,7 @@ Clustering makeClustering(TableMetadata table, String...clusteringKeys) { ByteBuffer[] values = new ByteBuffer[clusteringKeys.length]; for (int index = 0; index < table.comparator.size(); index++) - values[index] = table.comparator.subtype(index).fromString(clusteringKeys[index]); + values[index] = table.comparator.subtype(index).decomposeUntyped(clusteringKeys[index]); clustering = Clustering.make(values); } return clustering; diff --git a/test/unit/org/apache/cassandra/index/sai/utils/IndexInputLeakDetector.java b/test/unit/org/apache/cassandra/index/sai/utils/IndexInputLeakDetector.java index 46643255b43b..732556015c9d 100644 --- a/test/unit/org/apache/cassandra/index/sai/utils/IndexInputLeakDetector.java +++ b/test/unit/org/apache/cassandra/index/sai/utils/IndexInputLeakDetector.java @@ -29,6 +29,7 @@ import org.apache.cassandra.index.sai.disk.io.TrackingIndexFileUtils; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.util.SequentialWriterOption; +import org.apache.cassandra.schema.TableMetadata; import static org.junit.Assert.assertTrue; @@ -44,6 +45,14 @@ public IndexDescriptor newIndexDescriptor(Descriptor descriptor, SequentialWrite return IndexDescriptor.empty(descriptor); } + public IndexDescriptor newIndexDescriptor(Descriptor descriptor, TableMetadata metadata, SequentialWriterOption sequentialWriterOption) + { + TrackingIndexFileUtils trackingIndexFileUtils = new TrackingIndexFileUtils(sequentialWriterOption); + trackedIndexFileUtils.add(trackingIndexFileUtils); + IndexFileUtils.instance = trackingIndexFileUtils; + return IndexDescriptor.empty(descriptor, metadata); + } + @Override protected void afterIfSuccessful() { diff --git a/test/unit/org/apache/cassandra/index/sai/utils/RowAwarePrimaryKeyTest.java b/test/unit/org/apache/cassandra/index/sai/utils/RowAwarePrimaryKeyTest.java index 44fb2d745faf..ddb65ddf94a4 100644 --- a/test/unit/org/apache/cassandra/index/sai/utils/RowAwarePrimaryKeyTest.java +++ b/test/unit/org/apache/cassandra/index/sai/utils/RowAwarePrimaryKeyTest.java @@ -34,11 +34,10 @@ public void singlePartitionTest() int rows = nextInt(10, 100); PrimaryKey[] keys = new PrimaryKey[rows]; for (int index = 0; index < rows; index++) - keys[index] = factory.create(makeKey(simplePartition, Integer.toString(index)), Clustering.EMPTY); + keys[index] = factory.create(makeKey(simplePartition, index), Clustering.EMPTY); Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -53,7 +52,6 @@ public void compositePartitionTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -67,7 +65,7 @@ public void simplePartitonSingleClusteringAscTest() int clustering = 0; for (int index = 0; index < rows; index++) { - keys[index] = factory.create(makeKey(simplePartitionSingleClusteringAsc, Integer.toString(partition)), + keys[index] = factory.create(makeKey(simplePartitionSingleClusteringAsc, partition), makeClustering(simplePartitionSingleClusteringAsc, Integer.toString(clustering++))); if (clustering == 5) { @@ -78,7 +76,6 @@ public void simplePartitonSingleClusteringAscTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -93,7 +90,7 @@ public void simplePartitionMultipleClusteringAscTest() int clustering2 = 0; for (int index = 0; index < rows; index++) { - keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringAsc, Integer.toString(partition)), + keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringAsc, partition), makeClustering(simplePartitionMultipleClusteringAsc, Integer.toString(clustering1), Integer.toString(clustering2++))); if (clustering2 == 5) { @@ -109,7 +106,6 @@ public void simplePartitionMultipleClusteringAscTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -123,7 +119,7 @@ public void simplePartitonSingleClusteringDescTest() int clustering = 0; for (int index = 0; index < rows; index++) { - keys[index] = factory.create(makeKey(simplePartitionSingleClusteringDesc, Integer.toString(partition)), + keys[index] = factory.create(makeKey(simplePartitionSingleClusteringDesc, partition), makeClustering(simplePartitionSingleClusteringDesc, Integer.toString(clustering++))); if (clustering == 5) { @@ -134,7 +130,6 @@ public void simplePartitonSingleClusteringDescTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -149,7 +144,7 @@ public void simplePartitionMultipleClusteringDescTest() int clustering2 = 0; for (int index = 0; index < rows; index++) { - keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringDesc, Integer.toString(partition)), + keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringDesc, partition), makeClustering(simplePartitionMultipleClusteringDesc, Integer.toString(clustering1), Integer.toString(clustering2++))); if (clustering2 == 5) { @@ -165,7 +160,6 @@ public void simplePartitionMultipleClusteringDescTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -190,7 +184,6 @@ public void compositePartitionSingleClusteringAscTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -221,7 +214,6 @@ public void compositePartitionMultipleClusteringAscTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -246,7 +238,6 @@ public void compositePartitionSingleClusteringDescTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -277,7 +268,6 @@ public void compositePartitionMultipleClusteringDescTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -292,7 +282,7 @@ public void simplePartitionMultipleClusteringMixedTest() int clustering2 = 0; for (int index = 0; index < rows; index++) { - keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringMixed, Integer.toString(partition)), + keys[index] = factory.create(makeKey(simplePartitionMultipleClusteringMixed, partition), makeClustering(simplePartitionMultipleClusteringMixed, Integer.toString(clustering1), Integer.toString(clustering2++))); if (clustering2 == 5) { @@ -308,7 +298,6 @@ public void simplePartitionMultipleClusteringMixedTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -339,7 +328,6 @@ public void compositePartitionMultipleClusteringMixedTest() Arrays.sort(keys); - byteComparisonTests(factory, keys); compareToAndEqualsTests(factory, keys); } @@ -361,22 +349,4 @@ private void compareToAndEqualsTests(PrimaryKey.Factory factory, PrimaryKey... k } } } - - private void byteComparisonTests(PrimaryKey.Factory factory, PrimaryKey... keys) - { - for (int index = 0; index < keys.length - 1; index++) - { - PrimaryKey key = keys[index]; - PrimaryKey tokenOnlyKey = factory.createTokenOnly(key.token()); - assertByteComparison(tokenOnlyKey, key, -1); - assertByteComparison(key, key, 0); - assertByteComparison(tokenOnlyKey, tokenOnlyKey, 0); - - for (int comparisonIndex = index + 1; comparisonIndex < keys.length; comparisonIndex++) - { - assertByteComparison(key, keys[comparisonIndex], -1); - assertByteComparison(tokenOnlyKey, keys[comparisonIndex], -1); - } - } - } } diff --git a/test/unit/org/apache/cassandra/index/sai/utils/SaiRandomizedTest.java b/test/unit/org/apache/cassandra/index/sai/utils/SaiRandomizedTest.java index b2b7ace5de8f..452030807fc0 100644 --- a/test/unit/org/apache/cassandra/index/sai/utils/SaiRandomizedTest.java +++ b/test/unit/org/apache/cassandra/index/sai/utils/SaiRandomizedTest.java @@ -22,6 +22,7 @@ import java.util.Random; import com.google.common.base.Preconditions; + import org.apache.cassandra.io.util.FileUtils; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -35,11 +36,10 @@ import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.index.sai.disk.PostingList; import org.apache.cassandra.index.sai.disk.format.IndexDescriptor; -import org.apache.cassandra.io.compress.BufferType; import org.apache.cassandra.io.sstable.Descriptor; import org.apache.cassandra.io.sstable.SequenceBasedSSTableId; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.SequentialWriterOption; +import org.apache.cassandra.schema.TableMetadata; @ThreadLeakScope(ThreadLeakScope.Scope.NONE) public class SaiRandomizedTest extends RandomizedTest @@ -74,7 +74,7 @@ private static void restoreUncaughtExceptionHandler() Thread.setDefaultUncaughtExceptionHandler(handler); } - private static IndexInputLeakDetector indexInputLeakDetector; + private static final IndexInputLeakDetector indexInputLeakDetector; protected static TemporaryFolder temporaryFolder; @@ -88,13 +88,17 @@ public IndexDescriptor newIndexDescriptor() throws IOException randomSimpleString(5, 13), randomSimpleString(3, 17), new SequenceBasedSSTableId(randomIntBetween(0, 128))), - SequentialWriterOption.newBuilder() - .bufferSize(randomIntBetween(17, 1 << 13)) - .bufferType(randomBoolean() ? BufferType.ON_HEAP : BufferType.OFF_HEAP) - .trickleFsync(randomBoolean()) - .trickleFsyncByteInterval(nextInt(1 << 10, 1 << 16)) - .finishOnClose(true) - .build()); + IndexFileUtils.defaultWriterOption); + } + + public static IndexDescriptor newClusteringIndexDescriptor(TableMetadata metadata) throws IOException + { + return indexInputLeakDetector.newIndexDescriptor(new Descriptor(new File(temporaryFolder.newFolder()), + randomSimpleString(5, 13), + randomSimpleString(3, 17), + new SequenceBasedSSTableId(randomIntBetween(0, 128))), + metadata, + IndexFileUtils.defaultWriterOption); } public String newIndex() @@ -152,7 +156,7 @@ public static int randomIntBetween(int min, int max) { if (min < 0) throw new IllegalArgumentException("min must be >= 0: " + min); if (min > max) throw new IllegalArgumentException("max must be >= min: " + min + ", " + max); - return min == max ? min : (int) randomDoubleBetween((double) min, (double) max); + return min == max ? min : (int) randomDoubleBetween(min, max); } public static long randomLongBetween(long min, long max)