Numeric on-disk index write and search
Includes:
- The disk/v1/kdtree package containing the
kdtree writer and reader
- The implementation code to tie these into
the existing read and write paths. The main parts
of this are the NumericIndexWriter and the
NumericIndexSegmentSearcher
- Additional testing for the new code
patch by Mike Adamson; reviewed by Caleb Rackliffe and Andres de la Peña for CASSANDRA-18067
Co-authored-by: Mike Adamson <madamson@datastax.com>
Co-authored-by: Caleb Rackliffe <calebrackliffe@gmail.com>
Co-authored-by: Piotr Kołaczkowski <pkolaczk@gmail.com>
Co-authored-by: Jason Rutherglen <jason.rutherglen@gmail.com>
Co-authored-by: Zhao Yang <zhaoyangsingapore@gmail.com>
diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
index 09c5535..5d5c142 100644
--- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
+++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java
@@ -419,11 +419,23 @@
// SAI specific properties
/** Controls the maximum number of index query intersections that will take part in a query */
- SAI_INTERSECTION_CLAUSE_LIMIT("cassandra.sai.intersection.clause.limit", "2"),
+ SAI_INTERSECTION_CLAUSE_LIMIT("cassandra.sai.intersection_clause_limit", "2"),
/** Latest version to be used for SAI index writing */
SAI_LATEST_VERSION("cassandra.sai.latest_version", "aa"),
SAI_MAX_FROZEN_TERM_SIZE("cassandra.sai.max_frozen_term_size_kb", "5"),
SAI_MAX_STRING_TERM_SIZE("cassandra.sai.max_string_term_size_kb", "1"),
+
+ /** Minimum number of reachable leaves for a given node to be eligible for an auxiliary posting list */
+ SAI_MINIMUM_POSTINGS_LEAVES("cassandra.sai.minimum_postings_leaves", "64"),
+
+ /**
+ * Skip, or the sampling interval, for selecting a balanced tree level that is eligible for an auxiliary posting list.
+ * Sampling starts from 0, but balanced tree root node is at level 1. For skip = 4, eligible levels are 4, 8, 12, etc. (no
+ * level 0, because there is no node at level 0).
+ */
+ SAI_POSTINGS_SKIP("cassandra.sai.postings_skip", "3"),
+
+ SAI_TEST_BALANCED_TREE_DEBUG_ENABLED("cassandra.sai.test.balanced_tree_debug_enabled", "false"),
SAI_TEST_DISABLE_TIMEOUT("cassandra.sai.test.disable.timeout", "false"),
SCHEMA_PULL_INTERVAL_MS("cassandra.schema_pull_interval_ms", "60000"),
diff --git a/src/java/org/apache/cassandra/db/ClusteringComparator.java b/src/java/org/apache/cassandra/db/ClusteringComparator.java
index d007c5d..2949130 100644
--- a/src/java/org/apache/cassandra/db/ClusteringComparator.java
+++ b/src/java/org/apache/cassandra/db/ClusteringComparator.java
@@ -353,6 +353,8 @@
{
ByteComparable.Version version = ByteComparable.Version.OSS50;
ByteSource.Peekable orderedBytes = ByteSource.peekable(comparable.asComparableBytes(version));
+ if (orderedBytes == null)
+ return null;
// First check for special cases (partition key only, static clustering) that can do without buffers.
int sep = orderedBytes.next();
diff --git a/src/java/org/apache/cassandra/index/sai/IndexContext.java b/src/java/org/apache/cassandra/index/sai/IndexContext.java
index 032f289..f166073 100644
--- a/src/java/org/apache/cassandra/index/sai/IndexContext.java
+++ b/src/java/org/apache/cassandra/index/sai/IndexContext.java
@@ -113,9 +113,11 @@
this.indexMetadata = indexMetadata;
this.memtableIndexManager = indexMetadata == null ? null : new MemtableIndexManager(this);
+
this.indexMetrics = indexMetadata == null ? null : new IndexMetrics(this);
this.viewManager = new IndexViewManager(this);
- this.columnQueryMetrics = new ColumnQueryMetrics.TrieIndexMetrics(this);
+ this.columnQueryMetrics = isLiteral() ? new ColumnQueryMetrics.TrieIndexMetrics(this)
+ : new ColumnQueryMetrics.BalancedTreeIndexMetrics(this);
// We currently only support the NoOpAnalyzer
this.indexAnalyzerFactory = AbstractAnalyzer.fromOptions(getValidator(), Collections.emptyMap());
diff --git a/src/java/org/apache/cassandra/index/sai/QueryContext.java b/src/java/org/apache/cassandra/index/sai/QueryContext.java
index 6bb85df..e9db787 100644
--- a/src/java/org/apache/cassandra/index/sai/QueryContext.java
+++ b/src/java/org/apache/cassandra/index/sai/QueryContext.java
@@ -48,10 +48,14 @@
public long rowsFiltered = 0;
public long trieSegmentsHit = 0;
-
public long triePostingsSkips = 0;
public long triePostingsDecodes = 0;
+ public long balancedTreePostingListsHit = 0;
+ public long balancedTreeSegmentsHit = 0;
+ public long balancedTreePostingsSkips = 0;
+ public long balancedTreePostingsDecodes = 0;
+
public boolean queryTimedOut = false;
public QueryContext(ReadCommand readCommand, long executionQuotaMs)
diff --git a/src/java/org/apache/cassandra/index/sai/README.md b/src/java/org/apache/cassandra/index/sai/README.md
index 45cdfee..6be0fa9 100644
--- a/src/java/org/apache/cassandra/index/sai/README.md
+++ b/src/java/org/apache/cassandra/index/sai/README.md
@@ -25,8 +25,8 @@
architectural character (and even some actual code), but makes significant improvements in a number of areas:
- The on-disk/SSTable index formats for both string and numeric data have been completely replaced. Strings are indexed
- on disk using our proprietary on-disk byte-ordered trie data structure, while numeric types are indexed using Lucene's
- balanced kd-tree.
+ on disk using our proprietary on-disk byte-ordered trie data structure, while numeric types are indexed using a
+ balanced tree.
- While indexes continue to be managed at the column level from the user's perspective, the storage design at the column
index level is row-based, with related offset and token information stored only once at the SSTable level. This
drastically reduces our on-disk footprint when several columns are indexed on the same table.
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 4d802cc..b1b2b4a 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
@@ -39,6 +39,12 @@
META("Meta"),
/**
+ * Balanced tree written by {@code BlockBalancedTreeWriter} indexes mappings of term to one or more segment row IDs
+ * (segment row ID = SSTable row ID - segment row ID offset).
+ */
+ BALANCED_TREE("BalancedTree"),
+
+ /**
* Term dictionary written by {@link TrieTermsDictionaryWriter} stores mappings of term and
* file pointer to posting block on posting file.
*/
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
index 6a497fb..da57ac2 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/MemtableIndexWriter.java
@@ -35,6 +35,8 @@
import org.apache.cassandra.index.sai.disk.RowMapping;
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.bbtree.BlockBalancedTreeIterator;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
import org.apache.cassandra.index.sai.disk.v1.trie.LiteralIndexWriter;
import org.apache.cassandra.index.sai.memory.MemtableIndex;
@@ -110,7 +112,7 @@
try (MemtableTermsIterator terms = new MemtableTermsIterator(memtable.getMinTerm(), memtable.getMaxTerm(), iterator))
{
- long cellCount = flush(minKey, maxKey, indexContext.getValidator(), terms);
+ long cellCount = flush(minKey, maxKey, indexContext.getValidator(), terms, rowMapping.maxSSTableRowId);
indexDescriptor.createComponentOnDisk(IndexComponent.COLUMN_COMPLETION_MARKER, indexContext);
@@ -136,10 +138,11 @@
private long flush(DecoratedKey minKey,
DecoratedKey maxKey,
AbstractType<?> termComparator,
- MemtableTermsIterator terms) throws IOException
+ MemtableTermsIterator terms,
+ long maxSSTableRowId) throws IOException
{
- long numRows = 0;
- SegmentMetadata.ComponentMetadataMap indexMetas = null;
+ long numRows;
+ SegmentMetadata.ComponentMetadataMap indexMetas;
if (TypeUtil.isLiteral(termComparator))
{
@@ -149,6 +152,15 @@
numRows = writer.getPostingsCount();
}
}
+ else
+ {
+ NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ indexContext,
+ TypeUtil.fixedSizeOf(termComparator),
+ maxSSTableRowId);
+ indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(terms, termComparator));
+ numRows = writer.getValueCount();
+ }
// If no rows were written we need to delete any created column index components
// so that the index is correctly identified as being empty (only having a completion marker)
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java b/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
index 7d129ba..37c0bbd 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/PerColumnIndexFiles.java
@@ -25,7 +25,6 @@
import org.apache.cassandra.index.sai.IndexContext;
import org.apache.cassandra.index.sai.disk.format.IndexComponent;
import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
-import org.apache.cassandra.index.sai.utils.TypeUtil;
import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.io.util.FileUtils;
@@ -45,11 +44,16 @@
{
this.indexDescriptor = indexDescriptor;
this.indexContext = indexContext;
- if (TypeUtil.isLiteral(indexContext.getValidator()))
+ if (indexContext.isLiteral())
{
files.put(IndexComponent.POSTING_LISTS, indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext));
files.put(IndexComponent.TERMS_DATA, indexDescriptor.createPerIndexFileHandle(IndexComponent.TERMS_DATA, indexContext));
}
+ else
+ {
+ files.put(IndexComponent.BALANCED_TREE, indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext));
+ files.put(IndexComponent.POSTING_LISTS, indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext));
+ }
}
public FileHandle termsData()
@@ -62,6 +66,11 @@
return getFile(IndexComponent.POSTING_LISTS);
}
+ public FileHandle balancedTree()
+ {
+ return getFile(IndexComponent.BALANCED_TREE);
+ }
+
@SuppressWarnings({"resource", "RedundantSuppression"})
private FileHandle getFile(IndexComponent indexComponent)
{
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
index 1319006..1f9c853 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/RowAwarePrimaryKeyMap.java
@@ -188,6 +188,9 @@
: clusteringComparator.clusteringFromByteComparable(ByteBufferAccessor.instance,
v -> ByteSourceInverse.nextComponentSource(peekable));
+ if (clustering == null)
+ clustering = Clustering.EMPTY;
+
return primaryKeyFactory.create(partitionKey, clustering);
}
catch (IOException e)
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
index eeb94d1..9727816 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/SSTableIndexWriter.java
@@ -327,7 +327,9 @@
private SegmentBuilder newSegmentBuilder()
{
- SegmentBuilder builder = new SegmentBuilder.RAMStringSegmentBuilder(indexContext.getValidator(), limiter);
+ SegmentBuilder builder = TypeUtil.isLiteral(indexContext.getValidator())
+ ? new SegmentBuilder.RAMStringSegmentBuilder(indexContext.getValidator(), limiter)
+ : new SegmentBuilder.BlockBalancedTreeSegmentBuilder(indexContext.getValidator(), limiter);
long globalBytesUsed = limiter.increment(builder.totalBytesAllocated());
logger.debug(indexContext.logMessage("Created new segment builder while flushing SSTable {}. Global segment memory usage now at {}."),
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 189d9b6..830712b 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
@@ -57,16 +57,21 @@
@VisibleForTesting
public static final Set<IndexComponent> PER_SSTABLE_COMPONENTS = EnumSet.of(IndexComponent.GROUP_COMPLETION_MARKER,
- IndexComponent.GROUP_META,
- IndexComponent.TOKEN_VALUES,
- IndexComponent.PRIMARY_KEY_TRIE,
- IndexComponent.PRIMARY_KEY_BLOCKS,
- IndexComponent.PRIMARY_KEY_BLOCK_OFFSETS);
+ IndexComponent.GROUP_META,
+ IndexComponent.TOKEN_VALUES,
+ IndexComponent.PRIMARY_KEY_TRIE,
+ IndexComponent.PRIMARY_KEY_BLOCKS,
+ IndexComponent.PRIMARY_KEY_BLOCK_OFFSETS);
@VisibleForTesting
public static final Set<IndexComponent> LITERAL_COMPONENTS = EnumSet.of(IndexComponent.COLUMN_COMPLETION_MARKER,
- IndexComponent.META,
- IndexComponent.TERMS_DATA,
- IndexComponent.POSTING_LISTS);
+ IndexComponent.META,
+ IndexComponent.TERMS_DATA,
+ IndexComponent.POSTING_LISTS);
+ @VisibleForTesting
+ public static final Set<IndexComponent> NUMERIC_COMPONENTS = EnumSet.of(IndexComponent.COLUMN_COMPLETION_MARKER,
+ IndexComponent.META,
+ IndexComponent.BALANCED_TREE,
+ IndexComponent.POSTING_LISTS);
/**
* Global limit on heap consumed by all index segment building that occurs outside the context of Memtable flush.
@@ -225,7 +230,7 @@
@Override
public Set<IndexComponent> perColumnIndexComponents(IndexContext indexContext)
{
- return LITERAL_COMPONENTS;
+ return indexContext.isLiteral() ? LITERAL_COMPONENTS : NUMERIC_COMPONENTS;
}
@Override
@@ -239,7 +244,8 @@
@Override
public int openFilesPerColumnIndex(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/bbtree/BlockBalancedTreeIterator.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIterator.java
new file mode 100644
index 0000000..a650784
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIterator.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * A helper class for mapping disparate iterators to a single iterator for consumption by the
+ * {@link BlockBalancedTreeWriter}
+ */
+public interface BlockBalancedTreeIterator extends Iterator<Pair<byte[], PostingList>>
+{
+ static BlockBalancedTreeIterator fromTermsIterator(final TermsIterator termsIterator, AbstractType<?> termsComparator)
+ {
+ return new BlockBalancedTreeIterator()
+ {
+ final byte[] scratch = new byte[TypeUtil.fixedSizeOf(termsComparator)];
+
+ @Override
+ public boolean hasNext()
+ {
+ return termsIterator.hasNext();
+ }
+
+ @Override
+ public Pair<byte[], PostingList> next()
+ {
+ ByteComparable term = termsIterator.next();
+ ByteSourceInverse.copyBytes(term.asComparableBytes(ByteComparable.Version.OSS50), scratch);
+ return Pair.create(scratch, termsIterator.postings());
+ }
+ };
+ }
+
+ static BlockBalancedTreeIterator fromTrieIterator(Iterator<Map.Entry<ByteComparable, PackedLongValues.Builder>> iterator, int bytesPerValue)
+ {
+ return new BlockBalancedTreeIterator()
+ {
+ final byte[] scratch = new byte[bytesPerValue];
+
+ @Override
+ public boolean hasNext()
+ {
+ return iterator.hasNext();
+ }
+
+ @Override
+ public Pair<byte[], PostingList> next()
+ {
+ Map.Entry<ByteComparable, PackedLongValues.Builder> entry = iterator.next();
+ ByteSourceInverse.copyBytes(entry.getKey().asComparableBytes(ByteComparable.Version.OSS50), scratch);
+ PackedLongValues postings = entry.getValue().build();
+ PackedLongValues.Iterator postingsIterator = postings.iterator();
+ return Pair.create(scratch, new PostingList() {
+ @Override
+ public long nextPosting()
+ {
+ if (postingsIterator.hasNext())
+ return postingsIterator.next();
+ return END_OF_STREAM;
+ }
+
+ @Override
+ public long size()
+ {
+ return postings.size();
+ }
+
+ @Override
+ public long advance(long targetRowID)
+ {
+ throw new UnsupportedOperationException();
+ }
+ });
+ }
+ };
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java
new file mode 100644
index 0000000..4b87f46
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsIndex.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+
+import com.carrotsearch.hppc.IntLongHashMap;
+import com.carrotsearch.hppc.IntLongMap;
+import org.apache.cassandra.index.sai.disk.io.IndexInputReader;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.RandomAccessReader;
+
+import static org.apache.cassandra.index.sai.disk.v1.SAICodecUtils.validate;
+
+/**
+ * Mapping between node ID and an offset to its auxiliary posting list (containing every row id from all leaves
+ * reachable from that node. See {@link BlockBalancedTreePostingsWriter}).
+ */
+class BlockBalancedTreePostingsIndex
+{
+ private final int size;
+ public final IntLongMap index = new IntLongHashMap();
+
+ BlockBalancedTreePostingsIndex(FileHandle postingsFileHandle, long filePosition) throws IOException
+ {
+ try (RandomAccessReader reader = postingsFileHandle.createReader();
+ IndexInputReader input = IndexInputReader.create(reader))
+ {
+ validate(input);
+ input.seek(filePosition);
+
+ size = input.readVInt();
+
+ for (int x = 0; x < size; x++)
+ {
+ final int node = input.readVInt();
+ final long filePointer = input.readVLong();
+
+ index.put(node, filePointer);
+ }
+ }
+ }
+
+ /**
+ * Returns <tt>true</tt> if given node ID has an auxiliary posting list.
+ */
+ boolean exists(int nodeID)
+ {
+ return index.containsKey(nodeID);
+ }
+
+ /**
+ * Returns an offset within the balanced tree postings file to the begining of the blocks summary of given node's auxiliary
+ * posting list.
+ *
+ * @throws IllegalArgumentException when given nodeID doesn't have an auxiliary posting list. Check first with
+ * {@link #exists(int)}
+ */
+ long getPostingsFilePointer(int nodeID)
+ {
+ return index.get(nodeID);
+ }
+
+ int size()
+ {
+ return size;
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java
new file mode 100644
index 0000000..6ba6894
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriter.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.disk.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.postings.MergePostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PackedLongsPostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsWriter;
+import org.apache.cassandra.index.sai.postings.PeekablePostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+/**
+ * Writes leaf postings and auxiliary posting lists for bbtree nodes. If a node has a posting list attached,
+ * it will contain every row id from all leaves reachable from that node.
+ * <p>
+ * Writer is stateful, because it needs to collect data from the balanced tree data structure first to find set of eligible
+ * nodes and leaf nodes reachable from them.
+ * <p>
+ * The leaf blocks are written in value order (in the order we pass them to the {@link BlockBalancedTreeWriter}).
+ * This allows us to skip reading the leaves, instead just order leaf blocks by their offset in the index file,
+ * and correlate them with buffered posting lists.
+ */
+@NotThreadSafe
+public class BlockBalancedTreePostingsWriter implements BlockBalancedTreeWalker.TraversalCallback
+{
+ private static final Logger logger = LoggerFactory.getLogger(BlockBalancedTreePostingsWriter.class);
+
+ private final TreeMap<Long, Integer> leafOffsetToNodeID = new TreeMap<>(Long::compareTo);
+ private final Multimap<Integer, Integer> nodeToChildLeaves = HashMultimap.create();
+
+ /**
+ * Minimum number of reachable leaves for a given node to be eligible for an auxiliary posting list.
+ */
+ private final int minimumPostingsLeaves;
+ /**
+ * Skip, or the sampling interval, for selecting a balanced tree level that is eligible for an auxiliary posting list.
+ * Sampling starts from 0, but the balanced tree root node is at level 1. For skip = 4, eligible levels are 4, 8, 12, etc. (no
+ * level 0, because there is no node at level 0).
+ */
+ private final int postingsSkip;
+
+ int numNonLeafPostings = 0;
+ int numLeafPostings = 0;
+
+ public BlockBalancedTreePostingsWriter()
+ {
+ minimumPostingsLeaves = CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.getInt();
+ postingsSkip = CassandraRelevantProperties.SAI_POSTINGS_SKIP.getInt();
+ }
+
+ /**
+ * Called when a leaf node is hit as we traverse the packed index.
+ *
+ * @param leafNodeID the current leaf node ID in the packed inded
+ * @param leafBlockFP the file pointer to the on-disk leaf block
+ * @param pathToRoot the path to the root leaf above this leaf. Contains all the intermediate leaf node IDs.
+ */
+ @Override
+ public void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot)
+ {
+ checkArgument(!pathToRoot.containsInt(leafNodeID));
+ checkArgument(pathToRoot.isEmpty() || leafNodeID > pathToRoot.get(pathToRoot.size() - 1));
+
+ leafOffsetToNodeID.put(leafBlockFP, leafNodeID);
+ for (int i = 0; i < pathToRoot.size(); i++)
+ {
+ int level = i + 1;
+ if (isLevelEligibleForPostingList(level))
+ {
+ int nodeID = pathToRoot.get(i);
+ nodeToChildLeaves.put(nodeID, leafNodeID);
+ }
+ }
+ }
+
+ /**
+ * Writes merged posting lists for eligible internal nodes and leaf postings for each leaf in the tree.
+ * The merged postings list for an internal node contains all postings from the postings lists of leaf nodes
+ * in the subtree rooted at that node.
+ * <p>
+ * After writing out the postings, it writes a map of node ID -> postings file pointer for all
+ * nodes with an attached postings list. It then returns the file pointer to this map.
+ */
+ public long finish(IndexOutputWriter out, List<PackedLongValues> leafPostings, IndexContext indexContext) throws IOException
+ {
+ checkState(leafPostings.size() == leafOffsetToNodeID.size(),
+ "Expected equal number of postings lists (%s) and leaf offsets (%s).",
+ leafPostings.size(), leafOffsetToNodeID.size());
+
+ try (PostingsWriter postingsWriter = new PostingsWriter(out))
+ {
+ Iterator<PackedLongValues> postingsIterator = leafPostings.iterator();
+ Map<Integer, PackedLongValues> leafToPostings = new HashMap<>();
+ leafOffsetToNodeID.forEach((fp, nodeID) -> leafToPostings.put(nodeID, postingsIterator.next()));
+
+ long postingsRamBytesUsed = leafPostings.stream()
+ .mapToLong(PackedLongValues::ramBytesUsed)
+ .sum();
+
+ List<Integer> internalNodeIDs = nodeToChildLeaves.keySet()
+ .stream()
+ .filter(i -> nodeToChildLeaves.get(i).size() >= minimumPostingsLeaves)
+ .collect(Collectors.toList());
+
+ Collection<Integer> leafNodeIDs = leafOffsetToNodeID.values();
+
+ logger.debug(indexContext.logMessage("Writing posting lists for {} internal and {} leaf balanced tree nodes. Leaf postings memory usage: {}."),
+ internalNodeIDs.size(), leafNodeIDs.size(), FBUtilities.prettyPrintMemory(postingsRamBytesUsed));
+
+ long startFP = out.getFilePointer();
+ Stopwatch flushTime = Stopwatch.createStarted();
+ TreeMap<Integer, Long> nodeIDToPostingsFilePointer = new TreeMap<>();
+ PriorityQueue<PeekablePostingList> postingLists = new PriorityQueue<>(minimumPostingsLeaves, Comparator.comparingLong(PeekablePostingList::peek));
+ for (int nodeID : Iterables.concat(internalNodeIDs, leafNodeIDs))
+ {
+ Collection<Integer> leaves = nodeToChildLeaves.get(nodeID);
+
+ if (leaves.isEmpty())
+ {
+ leaves = Collections.singletonList(nodeID);
+ numLeafPostings++;
+ }
+ else
+ {
+ numNonLeafPostings++;
+ }
+
+ for (Integer leaf : leaves)
+ postingLists.add(PeekablePostingList.makePeekable(new PackedLongsPostingList(leafToPostings.get(leaf))));
+
+ try (PostingList mergedPostingList = MergePostingList.merge(postingLists))
+ {
+ long postingFilePosition = postingsWriter.write(mergedPostingList);
+ // During compaction, we could end up with an empty postings due to deletions.
+ // The writer will return a fp of -1 if no postings were written.
+ if (postingFilePosition >= 0)
+ nodeIDToPostingsFilePointer.put(nodeID, postingFilePosition);
+ }
+ postingLists.clear();
+ }
+ flushTime.stop();
+ logger.debug(indexContext.logMessage("Flushed {} of posting lists for balanced tree nodes in {} ms."),
+ FBUtilities.prettyPrintMemory(out.getFilePointer() - startFP),
+ flushTime.elapsed(TimeUnit.MILLISECONDS));
+
+ long indexFilePointer = out.getFilePointer();
+ writeMap(nodeIDToPostingsFilePointer, out);
+ postingsWriter.complete();
+ return indexFilePointer;
+ }
+ }
+
+ private boolean isLevelEligibleForPostingList(int level)
+ {
+ return level > 1 && level % postingsSkip == 0;
+ }
+
+ private void writeMap(Map<Integer, Long> map, IndexOutput out) throws IOException
+ {
+ out.writeVInt(map.size());
+
+ for (Map.Entry<Integer, Long> e : map.entrySet())
+ {
+ out.writeVInt(e.getKey());
+ out.writeVLong(e.getValue());
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java
new file mode 100644
index 0000000..483fa5a
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueries.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.lucene.index.PointValues.Relation;
+
+public class BlockBalancedTreeQueries
+{
+ private static final BlockBalancedTreeReader.IntersectVisitor MATCH_ALL = new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ return true;
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ return Relation.CELL_INSIDE_QUERY;
+ }
+ };
+
+ public static BlockBalancedTreeReader.IntersectVisitor balancedTreeQueryFrom(Expression expression, int bytesPerValue)
+ {
+ if (expression.lower == null && expression.upper == null)
+ {
+ return MATCH_ALL;
+ }
+
+ Bound lower = null ;
+ if (expression.lower != null)
+ {
+ final byte[] lowerBound = toComparableBytes(bytesPerValue, expression.lower.value.encoded, expression.validator);
+ lower = new Bound(lowerBound, !expression.lower.inclusive);
+ }
+
+ Bound upper = null;
+ if (expression.upper != null)
+ {
+ final byte[] upperBound = toComparableBytes(bytesPerValue, expression.upper.value.encoded, expression.validator);
+ upper = new Bound(upperBound, !expression.upper.inclusive);
+ }
+
+ return new RangeQueryVisitor(lower, upper);
+ }
+
+ private static byte[] toComparableBytes(int bytesPerDim, ByteBuffer value, AbstractType<?> type)
+ {
+ byte[] buffer = new byte[TypeUtil.fixedSizeOf(type)];
+ assert buffer.length == bytesPerDim;
+ TypeUtil.toComparableBytes(value, type, buffer);
+ return buffer;
+ }
+
+ private static class Bound
+ {
+ private final byte[] bound;
+ private final boolean exclusive;
+
+ Bound(byte[] bound, boolean exclusive)
+ {
+ this.bound = bound;
+ this.exclusive = exclusive;
+ }
+
+ boolean smallerThan(byte[] packedValue)
+ {
+ int cmp = compareTo(packedValue);
+ return cmp < 0 || (cmp == 0 && exclusive);
+ }
+
+ boolean greaterThan(byte[] packedValue)
+ {
+ int cmp = compareTo(packedValue);
+ return cmp > 0 || (cmp == 0 && exclusive);
+ }
+
+ private int compareTo(byte[] packedValue)
+ {
+ return ByteArrayUtil.compareUnsigned(bound, 0, packedValue, 0, bound.length);
+ }
+ }
+
+ private static class RangeQueryVisitor implements BlockBalancedTreeReader.IntersectVisitor
+ {
+ private final Bound lower;
+ private final Bound upper;
+
+ private RangeQueryVisitor(Bound lower, Bound upper)
+ {
+ this.lower = lower;
+ this.upper = upper;
+ }
+
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ if (lower != null)
+ {
+ if (lower.greaterThan(packedValue))
+ {
+ // value is too low, in this dimension
+ return false;
+ }
+ }
+
+ if (upper != null)
+ {
+ return !upper.smallerThan(packedValue);
+ }
+
+ return true;
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ boolean crosses = false;
+
+ if (lower != null)
+ {
+ if (lower.greaterThan(maxPackedValue))
+ return Relation.CELL_OUTSIDE_QUERY;
+
+ crosses = lower.greaterThan(minPackedValue);
+ }
+
+ if (upper != null)
+ {
+ if (upper.smallerThan(minPackedValue))
+ return Relation.CELL_OUTSIDE_QUERY;
+
+ crosses |= upper.smallerThan(maxPackedValue);
+ }
+
+ return crosses ? Relation.CELL_CROSSES_QUERY : Relation.CELL_INSIDE_QUERY;
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java
new file mode 100644
index 0000000..dfea17e
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBuffer.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.util.concurrent.atomic.LongAdder;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.cassandra.db.memtable.TrieMemtable;
+import org.apache.cassandra.db.tries.InMemoryTrie;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * On-heap buffer for values that provides a sorted view of itself as a {@link BlockBalancedTreeIterator}.
+ */
+@NotThreadSafe
+public class BlockBalancedTreeRamBuffer
+{
+ private final InMemoryTrie<PackedLongValues.Builder> trie;
+ private final PostingsAccumulator postingsAccumulator;
+ private final int bytesPerValue;
+ private int numRows;
+
+ public BlockBalancedTreeRamBuffer(int bytesPerValue)
+ {
+ trie = new InMemoryTrie<>(TrieMemtable.BUFFER_TYPE);
+ postingsAccumulator = new PostingsAccumulator();
+ this.bytesPerValue = bytesPerValue;
+ }
+
+ public int numRows()
+ {
+ return numRows;
+ }
+
+ public long memoryUsed()
+ {
+ return trie.sizeOnHeap() + postingsAccumulator.heapAllocations();
+ }
+
+ public long add(int segmentRowId, byte[] value)
+ {
+ final long initialSizeOnHeap = trie.sizeOnHeap();
+ final long reducerHeapSize = postingsAccumulator.heapAllocations();
+
+ try
+ {
+ trie.putRecursive(v -> ByteSource.fixedLength(value), segmentRowId, postingsAccumulator);
+ }
+ catch (InMemoryTrie.SpaceExhaustedException e)
+ {
+ throw Throwables.unchecked(e);
+ }
+
+ numRows++;
+ return (trie.sizeOnHeap() - initialSizeOnHeap) + (postingsAccumulator.heapAllocations() - reducerHeapSize);
+ }
+
+ public BlockBalancedTreeIterator iterator()
+ {
+ return BlockBalancedTreeIterator.fromTrieIterator(trie.entrySet().iterator(), bytesPerValue);
+ }
+
+ private static class PostingsAccumulator implements InMemoryTrie.UpsertTransformer<PackedLongValues.Builder, Integer>
+ {
+ private final LongAdder heapAllocations = new LongAdder();
+
+ @Override
+ public PackedLongValues.Builder apply(PackedLongValues.Builder existing, Integer rowID)
+ {
+ if (existing == null)
+ {
+ existing = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
+ heapAllocations.add(existing.ramBytesUsed());
+ }
+ long ramBefore = existing.ramBytesUsed();
+ existing.add(rowID);
+ heapAllocations.add(existing.ramBytesUsed() - ramBefore);
+ return existing;
+ }
+
+ long heapAllocations()
+ {
+ return heapAllocations.longValue();
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java
new file mode 100644
index 0000000..59271e3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReader.java
@@ -0,0 +1,427 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Comparator;
+import java.util.PriorityQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Stopwatch;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.exceptions.QueryCancelledException;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.io.IndexFileUtils;
+import org.apache.cassandra.index.sai.disk.io.SeekingRandomAccessInput;
+import org.apache.cassandra.index.sai.disk.v1.postings.FilteringPostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.MergePostingList;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PeekablePostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.DirectWriter;
+
+/**
+ * Handles intersection of a point or point range with a block balanced tree previously written with
+ * {@link BlockBalancedTreeWriter}.
+ */
+public class BlockBalancedTreeReader extends BlockBalancedTreeWalker implements Closeable
+{
+ private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ private static final Comparator<PeekablePostingList> COMPARATOR = Comparator.comparingLong(PeekablePostingList::peek);
+
+ private final IndexContext indexContext;
+ private final FileHandle postingsFile;
+ private final BlockBalancedTreePostingsIndex postingsIndex;
+ private final int leafOrderMapBitsRequired;
+ /**
+ * Performs a blocking read.
+ */
+ public BlockBalancedTreeReader(IndexContext indexContext,
+ FileHandle treeIndexFile,
+ long treeIndexRoot,
+ FileHandle postingsFile,
+ long treePostingsRoot) throws IOException
+ {
+ super(treeIndexFile, treeIndexRoot);
+ this.indexContext = indexContext;
+ this.postingsFile = postingsFile;
+ this.postingsIndex = new BlockBalancedTreePostingsIndex(postingsFile, treePostingsRoot);
+ leafOrderMapBitsRequired = DirectWriter.unsignedBitsRequired(state.maxPointsInLeafNode - 1);
+ }
+
+ public int getBytesPerValue()
+ {
+ return state.bytesPerValue;
+ }
+
+ public long getPointCount()
+ {
+ return state.valueCount;
+ }
+
+ @Override
+ public void close()
+ {
+ super.close();
+ FileUtils.closeQuietly(postingsFile);
+ }
+
+ @SuppressWarnings({"resource", "RedundantSuppression"})
+ public PostingList intersect(IntersectVisitor visitor, QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+ {
+ Relation relation = visitor.compare(state.minPackedValue, state.maxPackedValue);
+
+ if (relation == Relation.CELL_OUTSIDE_QUERY)
+ {
+ listener.onIntersectionEarlyExit();
+ return null;
+ }
+
+ listener.onSegmentHit();
+ IndexInput treeInput = IndexFileUtils.instance.openInput(treeIndexFile);
+ IndexInput postingsInput = IndexFileUtils.instance.openInput(postingsFile);
+ IndexInput postingsSummaryInput = IndexFileUtils.instance.openInput(postingsFile);
+ state.reset();
+
+ Intersection intersection = relation == Relation.CELL_INSIDE_QUERY
+ ? new Intersection(treeInput, postingsInput, postingsSummaryInput, listener, context)
+ : new FilteringIntersection(treeInput, postingsInput, postingsSummaryInput, visitor, listener, context);
+
+ return intersection.execute();
+ }
+
+ /**
+ * Synchronous intersection of a point or point range with a block balanced tree previously written
+ * with {@link BlockBalancedTreeWriter}.
+ */
+ private class Intersection
+ {
+ private final Stopwatch queryExecutionTimer = Stopwatch.createStarted();
+ final QueryContext context;
+
+ final IndexInput treeInput;
+ final IndexInput postingsInput;
+ final IndexInput postingsSummaryInput;
+ final QueryEventListener.BalancedTreeEventListener listener;
+ final PriorityQueue<PeekablePostingList> postingLists;
+
+ Intersection(IndexInput treeInput, IndexInput postingsInput, IndexInput postingsSummaryInput,
+ QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+ {
+ this.treeInput = treeInput;
+ this.postingsInput = postingsInput;
+ this.postingsSummaryInput = postingsSummaryInput;
+ this.listener = listener;
+ this.context = context;
+ postingLists = new PriorityQueue<>(state.numLeaves, COMPARATOR);
+ }
+
+ public PostingList execute()
+ {
+ try
+ {
+ executeInternal();
+
+ FileUtils.closeQuietly(treeInput);
+
+ return mergePostings();
+ }
+ catch (Throwable t)
+ {
+ if (!(t instanceof QueryCancelledException))
+ logger.error(indexContext.logMessage("Balanced tree intersection failed on {}"), treeIndexFile.path(), t);
+
+ closeOnException();
+ throw Throwables.cleaned(t);
+ }
+ }
+
+ protected void executeInternal() throws IOException
+ {
+ collectPostingLists();
+ }
+
+ protected void closeOnException()
+ {
+ FileUtils.closeQuietly(treeInput);
+ FileUtils.closeQuietly(postingsInput);
+ FileUtils.closeQuietly(postingsSummaryInput);
+ }
+
+ protected PostingList mergePostings()
+ {
+ final long elapsedMicros = queryExecutionTimer.stop().elapsed(TimeUnit.MICROSECONDS);
+
+ listener.onIntersectionComplete(elapsedMicros, TimeUnit.MICROSECONDS);
+ listener.postingListsHit(postingLists.size());
+
+ if (postingLists.isEmpty())
+ {
+ FileUtils.closeQuietly(postingsInput);
+ FileUtils.closeQuietly(postingsSummaryInput);
+ return null;
+ }
+ else
+ {
+ if (logger.isTraceEnabled())
+ logger.trace(indexContext.logMessage("[{}] Intersection completed in {} microseconds. {} leaf and internal posting lists hit."),
+ treeIndexFile.path(), elapsedMicros, postingLists.size());
+ return MergePostingList.merge(postingLists, () -> FileUtils.close(postingsInput, postingsSummaryInput));
+ }
+ }
+
+ private void collectPostingLists() throws IOException
+ {
+ context.checkpoint();
+
+ // This will return true if the node is a child leaf that has postings or if there is postings for the
+ // entire subtree under a leaf
+ if (postingsIndex.exists(state.nodeID))
+ {
+ postingLists.add(initPostingReader(postingsIndex.getPostingsFilePointer(state.nodeID)));
+ return;
+ }
+
+ if (state.atLeafNode())
+ throw new CorruptIndexException(indexContext.logMessage(String.format("Leaf node %s does not have balanced tree postings.", state.nodeID)), "");
+
+ // Recurse on left subtree:
+ state.pushLeft();
+ collectPostingLists();
+ state.pop();
+
+ // Recurse on right subtree:
+ state.pushRight();
+ collectPostingLists();
+ state.pop();
+ }
+
+ @SuppressWarnings({"resource", "RedundantSuppression"})
+ private PeekablePostingList initPostingReader(long offset) throws IOException
+ {
+ final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset);
+ return PeekablePostingList.makePeekable(new PostingsReader(postingsInput, summary, listener.postingListEventListener()));
+ }
+ }
+
+ private class FilteringIntersection extends Intersection
+ {
+ private final IntersectVisitor visitor;
+ private final byte[] packedValue;
+ private final short[] origIndex;
+
+ FilteringIntersection(IndexInput treeInput, IndexInput postingsInput, IndexInput postingsSummaryInput,
+ IntersectVisitor visitor, QueryEventListener.BalancedTreeEventListener listener, QueryContext context)
+ {
+ super(treeInput, postingsInput, postingsSummaryInput, listener, context);
+ this.visitor = visitor;
+ this.packedValue = new byte[state.bytesPerValue];
+ this.origIndex = new short[state.maxPointsInLeafNode];
+ }
+
+ @Override
+ public void executeInternal() throws IOException
+ {
+ collectPostingLists(state.minPackedValue, state.maxPackedValue);
+ }
+
+ private void collectPostingLists(byte[] minPackedValue, byte[] maxPackedValue) throws IOException
+ {
+ context.checkpoint();
+
+ final Relation r = visitor.compare(minPackedValue, maxPackedValue);
+
+ // This value range is fully outside the query shape: stop recursing
+ if (r == Relation.CELL_OUTSIDE_QUERY)
+ return;
+
+ if (r == Relation.CELL_INSIDE_QUERY)
+ {
+ // This value range is fully inside the query shape: recursively add all points from this node without filtering
+ super.collectPostingLists();
+ return;
+ }
+
+ if (state.atLeafNode())
+ {
+ if (state.nodeExists())
+ filterLeaf();
+ return;
+ }
+
+ visitNode(minPackedValue, maxPackedValue);
+ }
+
+ private void filterLeaf() throws IOException
+ {
+ treeInput.seek(state.getLeafBlockFP());
+
+ int count = treeInput.readVInt();
+ int orderMapLength = treeInput.readVInt();
+ long orderMapPointer = treeInput.getFilePointer();
+
+ SeekingRandomAccessInput randomAccessInput = new SeekingRandomAccessInput(treeInput);
+ LongValues leafOrderMapReader = DirectReader.getInstance(randomAccessInput, leafOrderMapBitsRequired, orderMapPointer);
+ for (int index = 0; index < count; index++)
+ {
+ origIndex[index] = (short) Math.toIntExact(leafOrderMapReader.get(index));
+ }
+
+ // seek beyond the ordermap
+ treeInput.seek(orderMapPointer + orderMapLength);
+
+ FixedBitSet fixedBitSet = buildPostingsFilter(treeInput, count, visitor, origIndex);
+
+ if (postingsIndex.exists(state.nodeID) && fixedBitSet.cardinality() > 0)
+ {
+ long pointer = postingsIndex.getPostingsFilePointer(state.nodeID);
+ postingLists.add(initFilteringPostingReader(pointer, fixedBitSet));
+ }
+ }
+
+ void visitNode(byte[] minPackedValue, byte[] maxPackedValue) throws IOException
+ {
+ assert !state.atLeafNode() : "Cannot recurse down tree because nodeID " + state.nodeID + " is a leaf node";
+
+ byte[] splitValue = state.getSplitValue();
+
+ if (BlockBalancedTreeWriter.DEBUG)
+ {
+ // make sure cellMin <= splitValue <= cellMax:
+ assert ByteArrayUtil.compareUnsigned(minPackedValue, 0, splitValue, 0, state.bytesPerValue) <= 0 :"bytesPerValue=" + state.bytesPerValue;
+ assert ByteArrayUtil.compareUnsigned(maxPackedValue, 0, splitValue, 0, state.bytesPerValue) >= 0 : "bytesPerValue=" + state.bytesPerValue;
+ }
+
+ // Recurse on left subtree:
+ state.pushLeft();
+ collectPostingLists(minPackedValue, splitValue);
+ state.pop();
+
+ // Recurse on right subtree:
+ state.pushRight();
+ collectPostingLists(splitValue, maxPackedValue);
+ state.pop();
+ }
+
+ @SuppressWarnings({"resource", "RedundantSuppression"})
+ private PeekablePostingList initFilteringPostingReader(long offset, FixedBitSet filter) throws IOException
+ {
+ final PostingsReader.BlocksSummary summary = new PostingsReader.BlocksSummary(postingsSummaryInput, offset);
+ PostingsReader postingsReader = new PostingsReader(postingsInput, summary, listener.postingListEventListener());
+ return PeekablePostingList.makePeekable(new FilteringPostingList(filter, postingsReader));
+ }
+
+ private FixedBitSet buildPostingsFilter(IndexInput in, int count, IntersectVisitor visitor, short[] origIndex) throws IOException
+ {
+ int commonPrefixLength = readCommonPrefixLength(in);
+ return commonPrefixLength == state.bytesPerValue ? buildPostingsFilterForSingleValueLeaf(count, visitor, origIndex)
+ : buildPostingsFilterForMultiValueLeaf(commonPrefixLength, in, count, visitor, origIndex);
+ }
+
+ private FixedBitSet buildPostingsFilterForMultiValueLeaf(int commonPrefixLength,
+ IndexInput in,
+ int count,
+ IntersectVisitor visitor,
+ short[] origIndex) throws IOException
+ {
+ // the byte at `compressedByteOffset` is compressed using run-length compression,
+ // other suffix bytes are stored verbatim
+ int compressedByteOffset = commonPrefixLength;
+ commonPrefixLength++;
+ int i;
+
+ FixedBitSet fixedBitSet = new FixedBitSet(state.maxPointsInLeafNode);
+
+ for (i = 0; i < count; )
+ {
+ packedValue[compressedByteOffset] = in.readByte();
+ final int runLen = Byte.toUnsignedInt(in.readByte());
+ for (int j = 0; j < runLen; ++j)
+ {
+ in.readBytes(packedValue, commonPrefixLength, state.bytesPerValue - commonPrefixLength);
+ final int rowIDIndex = origIndex[i + j];
+ if (visitor.contains(packedValue))
+ fixedBitSet.set(rowIDIndex);
+ }
+ i += runLen;
+ }
+ if (i != count)
+ throw new CorruptIndexException(String.format("Expected %d sub-blocks but read %d.", count, i), in);
+
+ return fixedBitSet;
+ }
+
+ private FixedBitSet buildPostingsFilterForSingleValueLeaf(int count, IntersectVisitor visitor, final short[] origIndex)
+ {
+ FixedBitSet fixedBitSet = new FixedBitSet(state.maxPointsInLeafNode);
+
+ // All the values in the leaf are the same, so we only
+ // need to visit once then set the bits for the relevant indexes
+ if (visitor.contains(packedValue))
+ {
+ for (int i = 0; i < count; ++i)
+ fixedBitSet.set(origIndex[i]);
+ }
+ return fixedBitSet;
+ }
+
+ private int readCommonPrefixLength(IndexInput in) throws IOException
+ {
+ int prefixLength = in.readVInt();
+ if (prefixLength > 0)
+ in.readBytes(packedValue, 0, prefixLength);
+ return prefixLength;
+ }
+ }
+
+ /**
+ * We recurse the balanced tree, using a provided instance of this to guide the recursion.
+ */
+ public interface IntersectVisitor
+ {
+ /**
+ * Called for all values in a leaf cell that crosses the query. The consumer should scrutinize the packedValue
+ * to decide whether to accept it. Values are visited in increasing order, and in the case of ties,
+ * in increasing order by segment row ID.
+ */
+ boolean contains(byte[] packedValue);
+
+ /**
+ * Called for non-leaf cells to test how the cell relates to the query, to
+ * determine how to further recurse down the tree.
+ */
+ Relation compare(byte[] minPackedValue, byte[] maxPackedValue);
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java
new file mode 100644
index 0000000..ebfcd4c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWalker.java
@@ -0,0 +1,318 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.index.sai.disk.io.IndexInputReader;
+import org.apache.cassandra.index.sai.disk.v1.SAICodecUtils;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Base reader for a block balanced tree previously written with {@link BlockBalancedTreeWriter}.
+ * <p>
+ * Holds the index tree on heap and enables its traversal via {@link #traverse(TraversalCallback)}.
+ */
+public class BlockBalancedTreeWalker implements Closeable
+{
+ final FileHandle treeIndexFile;
+ final TraversalState state;
+
+ BlockBalancedTreeWalker(FileHandle treeIndexFile, long treeIndexRoot)
+ {
+ this.treeIndexFile = treeIndexFile;
+
+ try (RandomAccessReader reader = treeIndexFile.createReader();
+ IndexInput indexInput = IndexInputReader.create(reader))
+ {
+ SAICodecUtils.validate(indexInput);
+ indexInput.seek(treeIndexRoot);
+
+ state = new TraversalState(indexInput);
+ }
+ catch (Throwable t)
+ {
+ FileUtils.closeQuietly(treeIndexFile);
+ throw Throwables.unchecked(t);
+ }
+ }
+
+ public long memoryUsage()
+ {
+ return state.memoryUsage;
+ }
+
+ @Override
+ public void close()
+ {
+ FileUtils.closeQuietly(treeIndexFile);
+ }
+
+ void traverse(TraversalCallback callback)
+ {
+ state.reset();
+ traverse(callback, new IntArrayList());
+ }
+
+ private void traverse(TraversalCallback callback, IntArrayList pathToRoot)
+ {
+ if (state.atLeafNode())
+ {
+ // In the unbalanced case it's possible the left most node only has one child:
+ if (state.nodeExists())
+ {
+ callback.onLeaf(state.nodeID, state.getLeafBlockFP(), pathToRoot);
+ }
+ }
+ else
+ {
+ IntArrayList currentPath = new IntArrayList();
+ currentPath.addAll(pathToRoot);
+ currentPath.add(state.nodeID);
+
+ state.pushLeft();
+ traverse(callback, currentPath);
+ state.pop();
+
+ state.pushRight();
+ traverse(callback, currentPath);
+ state.pop();
+ }
+ }
+
+ interface TraversalCallback
+ {
+ void onLeaf(int leafNodeID, long leafBlockFP, IntArrayList pathToRoot);
+ }
+
+ /**
+ * This maintains the state for a traversal of the packed index. It is loaded once and can be resused
+ * by calling the reset method.
+ * <p>
+ * The packed index is a packed representation of a balanced tree and takes the form of a packed array of
+ * file pointer / split value pairs. Both the file pointers and split values are prefix compressed by tree level
+ * requiring us to maintain a stack of values for each level in the tree. The stack size is always the tree depth.
+ * <p>
+ * The tree is traversed by recursively following the left and then right subtrees under the current node. For the
+ * following tree (split values in square brackets):
+ * <pre>
+ * 1[16]
+ * / \
+ * / \
+ * 2[8] 3[24]
+ * / \ / \
+ * 4 5 6 7
+ * </pre>
+ * The traversal will be 1 -> 2 -> 4 -> 5 -> 3 -> 6 -> 7 with nodes 4, 5, 6 & 7 being leaf nodes.
+ * <p>
+ * Assuming the full range of values in the tree is 0 -> 32, the non-leaf nodes will represent the following
+ * values:
+ * <pre>
+ * 1[0-32]
+ * / \
+ * 2[0-16] 3[16-32]
+ * </pre>
+ */
+ final static class TraversalState
+ {
+ final int bytesPerValue;
+ final int numLeaves;
+ final int treeDepth;
+ final byte[] minPackedValue;
+ final byte[] maxPackedValue;
+ final long valueCount;
+ final int maxPointsInLeafNode;
+ final long memoryUsage;
+
+ // used to read the packed index byte[]
+ final ByteArrayDataInput dataInput;
+ // holds the minimum (left most) leaf block file pointer for each level we've recursed to:
+ final long[] leafBlockFPStack;
+ // holds the address, in the packed byte[] index, of the left-node of each level:
+ final int[] leftNodePositions;
+ // holds the address, in the packed byte[] index, of the right-node of each level:
+ final int[] rightNodePositions;
+ // holds the packed per-level split values; the run method uses this to save the cell min/max as it recurses:
+ final byte[][] splitValuesStack;
+
+ int nodeID;
+ int level;
+ @VisibleForTesting
+ int maxLevel;
+
+ TraversalState(DataInput dataInput) throws IOException
+ {
+ maxPointsInLeafNode = dataInput.readVInt();
+ bytesPerValue = dataInput.readVInt();
+
+ // Read index:
+ numLeaves = dataInput.readVInt();
+ assert numLeaves > 0;
+ treeDepth = dataInput.readVInt();
+ minPackedValue = new byte[bytesPerValue];
+ maxPackedValue = new byte[bytesPerValue];
+
+ dataInput.readBytes(minPackedValue, 0, bytesPerValue);
+ dataInput.readBytes(maxPackedValue, 0, bytesPerValue);
+
+ if (ByteArrayUtil.compareUnsigned(minPackedValue, 0, maxPackedValue, 0, bytesPerValue) > 0)
+ {
+ String message = String.format("Min packed value %s is > max packed value %s.",
+ new BytesRef(minPackedValue), new BytesRef(maxPackedValue));
+ throw new CorruptIndexException(message, dataInput);
+ }
+
+ valueCount = dataInput.readVLong();
+
+ int numBytes = dataInput.readVInt();
+ byte[] packedIndex = new byte[numBytes];
+ dataInput.readBytes(packedIndex, 0, numBytes);
+
+ nodeID = 1;
+ level = 0;
+ leafBlockFPStack = new long[treeDepth];
+ leftNodePositions = new int[treeDepth];
+ rightNodePositions = new int[treeDepth];
+ splitValuesStack = new byte[treeDepth][];
+
+ memoryUsage = ObjectSizes.sizeOfArray(packedIndex) +
+ ObjectSizes.sizeOfArray(minPackedValue) +
+ ObjectSizes.sizeOfArray(maxPackedValue) +
+ ObjectSizes.sizeOfArray(leafBlockFPStack) +
+ ObjectSizes.sizeOfArray(leftNodePositions) +
+ ObjectSizes.sizeOfArray(rightNodePositions) +
+ ObjectSizes.sizeOfArray(splitValuesStack) * bytesPerValue;
+
+ this.dataInput = new ByteArrayDataInput(packedIndex);
+ readNodeData(false);
+ }
+
+ public void reset()
+ {
+ nodeID = 1;
+ level = 0;
+ dataInput.setPosition(0);
+ }
+
+ public void pushLeft()
+ {
+ int nodePosition = leftNodePositions[level];
+ nodeID *= 2;
+ level++;
+ maxLevel = Math.max(maxLevel, level);
+ dataInput.setPosition(nodePosition);
+ readNodeData(true);
+ }
+
+ public void pushRight()
+ {
+ int nodePosition = rightNodePositions[level];
+ nodeID = nodeID * 2 + 1;
+ level++;
+ maxLevel = Math.max(maxLevel, level);
+ dataInput.setPosition(nodePosition);
+ readNodeData(false);
+ }
+
+ public void pop()
+ {
+ nodeID /= 2;
+ level--;
+ }
+
+ public boolean atLeafNode()
+ {
+ return nodeID >= numLeaves;
+ }
+
+ public boolean nodeExists()
+ {
+ return nodeID - numLeaves < numLeaves;
+ }
+
+ public long getLeafBlockFP()
+ {
+ return leafBlockFPStack[level];
+ }
+
+ public byte[] getSplitValue()
+ {
+ assert !atLeafNode();
+ return splitValuesStack[level];
+ }
+
+ private void readNodeData(boolean isLeft)
+ {
+ leafBlockFPStack[level] = level == 0 ? 0 : leafBlockFPStack[level - 1];
+
+ // read leaf block FP delta
+ if (!isLeft)
+ leafBlockFPStack[level] += dataInput.readVLong();
+
+ if (!atLeafNode())
+ {
+ // read prefix, firstDiffByteDelta encoded as int:
+ int code = dataInput.readVInt();
+ int prefix = code % (1 + bytesPerValue);
+ int suffix = bytesPerValue - prefix;
+
+ pushSplitValueStack();
+ if (suffix > 0)
+ {
+ int firstDiffByteDelta = code / (1 + bytesPerValue);
+ // If we are pushing to the left subtree then the delta will be negative
+ if (isLeft)
+ firstDiffByteDelta = -firstDiffByteDelta;
+ int oldByte = splitValuesStack[level][prefix] & 0xFF;
+ splitValuesStack[level][prefix] = (byte) (oldByte + firstDiffByteDelta);
+ dataInput.readBytes(splitValuesStack[level], prefix + 1, suffix - 1);
+ }
+
+ int leftNumBytes = nodeID * 2 < numLeaves ? dataInput.readVInt() : 0;
+
+ leftNodePositions[level] = dataInput.getPosition();
+ rightNodePositions[level] = leftNodePositions[level] + leftNumBytes;
+ }
+ }
+
+ private void pushSplitValueStack()
+ {
+ if (splitValuesStack[level] == null)
+ splitValuesStack[level] = new byte[bytesPerValue];
+ if (level == 0)
+ Arrays.fill(splitValuesStack[level], (byte) 0);
+ else
+ System.arraycopy(splitValuesStack[level - 1], 0, splitValuesStack[level], 0, bytesPerValue);
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java
new file mode 100644
index 0000000..f74a69c
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeWriter.java
@@ -0,0 +1,762 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.base.MoreObjects;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.index.sai.disk.ResettableByteBuffersIndexOutput;
+import org.apache.cassandra.index.sai.disk.v1.SAICodecUtils;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.utils.ByteArrayUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntroSorter;
+import org.apache.lucene.util.Sorter;
+import org.apache.lucene.util.bkd.BKDWriter;
+
+import static org.apache.cassandra.index.sai.postings.PostingList.END_OF_STREAM;
+
+/**
+ * This is a specialisation of the Lucene {@link BKDWriter} that only writes a single dimension
+ * balanced tree.
+ * <p>
+ * Recursively builds a block balanced tree to assign all incoming points to smaller
+ * and smaller rectangles (cells) until the number of points in a given
+ * rectangle is <= <code>maxPointsInLeafNode</code>. The tree is
+ * fully balanced, which means the leaf nodes will have between 50% and 100% of
+ * the requested <code>maxPointsInLeafNode</code>. Values that fall exactly
+ * on a cell boundary may be in either cell.
+ * <p>
+ * Visual representation of the disk format:
+ * <pre>
+ *
+ * +========+=======================================+==================+========+
+ * | HEADER | LEAF BLOCK LIST | BALANCED TREE | FOOTER |
+ * +========+================+=====+================+==================+========+
+ * | LEAF BLOCK (0) | ... | LEAF BLOCK (N) | VALUES PER LEAF |
+ * +----------------+-----+----------------+------------------|
+ * | ORDER INDEX | | BYTES PER VALUE |
+ * +----------------+ +------------------+
+ * | PREFIX | | NUMBER OF LEAVES |
+ * +----------------+ +------------------+
+ * | VALUES | | MINIMUM VALUE |
+ * +----------------+ +------------------+
+ * | MAXIMUM VALUE |
+ * +------------------+
+ * | TOTAL VALUES |
+ * +------------------+
+ * | INDEX TREE |
+ * +--------+---------+
+ * | LENGTH | BYTES |
+ * +--------+---------+
+ * </pre>
+ *
+ * <p>
+ * <b>NOTE</b>: This can write at most Integer.MAX_VALUE * <code>maxPointsInLeafNode</code> total points.
+ * <p>
+ * @see BKDWriter
+ */
+@NotThreadSafe
+public class BlockBalancedTreeWriter
+{
+ // Enable to check that values are added to the tree in correct order and within bounds
+ public static final boolean DEBUG = CassandraRelevantProperties.SAI_TEST_BALANCED_TREE_DEBUG_ENABLED.getBoolean();
+
+ // Default maximum number of point in each leaf block
+ public static final int DEFAULT_MAX_POINTS_IN_LEAF_NODE = 1024;
+
+ private final int bytesPerValue;
+ private final int maxPointsInLeafNode;
+ private final byte[] minPackedValue;
+ private final byte[] maxPackedValue;
+ private long valueCount;
+
+ public BlockBalancedTreeWriter(int bytesPerValue, int maxPointsInLeafNode)
+ {
+ if (maxPointsInLeafNode <= 0)
+ throw new IllegalArgumentException("maxPointsInLeafNode must be > 0; got " + maxPointsInLeafNode);
+ if (maxPointsInLeafNode > ArrayUtil.MAX_ARRAY_LENGTH)
+ throw new IllegalArgumentException("maxPointsInLeafNode must be <= ArrayUtil.MAX_ARRAY_LENGTH (= " +
+ ArrayUtil.MAX_ARRAY_LENGTH + "); got " + maxPointsInLeafNode);
+
+ this.maxPointsInLeafNode = maxPointsInLeafNode;
+ this.bytesPerValue = bytesPerValue;
+
+ minPackedValue = new byte[bytesPerValue];
+ maxPackedValue = new byte[bytesPerValue];
+ }
+
+ public long getValueCount()
+ {
+ return valueCount;
+ }
+
+ public int getBytesPerValue()
+ {
+ return bytesPerValue;
+ }
+
+ public int getMaxPointsInLeafNode()
+ {
+ return maxPointsInLeafNode;
+ }
+
+ /**
+ * Write the sorted values from a {@link BlockBalancedTreeIterator}.
+ * <p>
+ * @param treeOutput The {@link IndexOutput} to write the balanced tree to
+ * @param values The {@link BlockBalancedTreeIterator} containing the values and rowIDs to be written
+ * @param callback The {@link Callback} used to record the leaf postings for each leaf
+ *
+ * @return The file pointer to the beginning of the balanced tree
+ */
+ public long write(IndexOutput treeOutput, BlockBalancedTreeIterator values, final Callback callback) throws IOException
+ {
+ SAICodecUtils.writeHeader(treeOutput);
+
+ LeafWriter leafWriter = new LeafWriter(treeOutput, callback);
+
+ while (values.hasNext())
+ {
+ Pair<byte[], PostingList> pair = values.next();
+ long segmentRowId;
+ while ((segmentRowId = pair.right.nextPosting()) != END_OF_STREAM)
+ leafWriter.add(pair.left, segmentRowId);
+ }
+
+ valueCount = leafWriter.finish();
+
+ long treeFilePointer = valueCount == 0 ? -1 : treeOutput.getFilePointer();
+
+ // There is only any point in writing the balanced tree if any values were added
+ if (treeFilePointer >= 0)
+ writeBalancedTree(treeOutput, maxPointsInLeafNode, leafWriter.leafBlockStartValues, leafWriter.leafBlockFilePointers);
+
+ SAICodecUtils.writeFooter(treeOutput);
+
+ return treeFilePointer;
+ }
+
+ private void writeBalancedTree(IndexOutput out, int countPerLeaf, List<byte[]> leafBlockStartValues, List<Long> leafBlockFilePointer) throws IOException
+ {
+ int numInnerNodes = leafBlockStartValues.size();
+ byte[] splitValues = new byte[(1 + numInnerNodes) * bytesPerValue];
+ int treeDepth = recurseBalanceTree(1, 0, numInnerNodes, 1, splitValues, leafBlockStartValues);
+ long[] leafBlockFPs = leafBlockFilePointer.stream().mapToLong(l -> l).toArray();
+ byte[] packedIndex = packIndex(leafBlockFPs, splitValues);
+
+ out.writeVInt(countPerLeaf);
+ out.writeVInt(bytesPerValue);
+
+ out.writeVInt(leafBlockFPs.length);
+ out.writeVInt(Math.min(treeDepth, leafBlockFPs.length));
+
+ out.writeBytes(minPackedValue, 0, bytesPerValue);
+ out.writeBytes(maxPackedValue, 0, bytesPerValue);
+
+ out.writeVLong(valueCount);
+
+ out.writeVInt(packedIndex.length);
+ out.writeBytes(packedIndex, 0, packedIndex.length);
+ }
+
+ /**
+ * This can, potentially, be removed in the future by CASSANDRA-18597
+ */
+ private int recurseBalanceTree(int nodeID, int offset, int count, int treeDepth, byte[] splitValues, List<byte[]> leafBlockStartValues)
+ {
+ if (count == 1)
+ {
+ treeDepth++;
+ // Leaf index node
+ System.arraycopy(leafBlockStartValues.get(offset), 0, splitValues, nodeID * bytesPerValue, bytesPerValue);
+ }
+ else if (count > 1)
+ {
+ treeDepth++;
+ // Internal index node: binary partition of count
+ int countAtLevel = 1;
+ int totalCount = 0;
+ while (true)
+ {
+ int countLeft = count - totalCount;
+ if (countLeft <= countAtLevel)
+ {
+ // This is the last level, possibly partially filled:
+ int lastLeftCount = Math.min(countAtLevel / 2, countLeft);
+ assert lastLeftCount >= 0;
+ int leftHalf = (totalCount - 1) / 2 + lastLeftCount;
+
+ int rootOffset = offset + leftHalf;
+
+ System.arraycopy(leafBlockStartValues.get(rootOffset), 0, splitValues, nodeID * bytesPerValue, bytesPerValue);
+
+ // TODO: we could optimize/specialize, when we know it's simply fully balanced binary tree
+ // under here, to save this while loop on each recursion
+
+ // Recurse left
+ int leftTreeDepth = recurseBalanceTree(2 * nodeID, offset, leftHalf, treeDepth, splitValues, leafBlockStartValues);
+
+ // Recurse right
+ int rightTreeDepth = recurseBalanceTree(2 * nodeID + 1, rootOffset + 1, count - leftHalf - 1, treeDepth, splitValues, leafBlockStartValues);
+ return Math.max(leftTreeDepth, rightTreeDepth);
+ }
+ totalCount += countAtLevel;
+ countAtLevel *= 2;
+ }
+ }
+ else
+ {
+ assert count == 0;
+ }
+ return treeDepth;
+ }
+
+ // Packs the two arrays, representing a balanced binary tree, into a compact byte[] structure.
+ private byte[] packIndex(long[] leafBlockFPs, byte[] splitValues) throws IOException
+ {
+ int numLeaves = leafBlockFPs.length;
+
+ // Possibly rotate the leaf block FPs, if the index is not a fully balanced binary tree (only happens
+ // if it was created by TreeWriter). In this case the leaf nodes may straddle the two bottom
+ // levels of the binary tree:
+ if (numLeaves > 1)
+ {
+ int levelCount = 2;
+ while (true)
+ {
+ if (numLeaves >= levelCount && numLeaves <= 2 * levelCount)
+ {
+ int lastLevel = 2 * (numLeaves - levelCount);
+ assert lastLevel >= 0;
+ if (lastLevel != 0)
+ {
+ // Last level is partially filled, so we must rotate the leaf FPs to match. We do this here, after loading
+ // at read-time, so that we can still delta code them on disk at write:
+ long[] newLeafBlockFPs = new long[numLeaves];
+ System.arraycopy(leafBlockFPs, lastLevel, newLeafBlockFPs, 0, leafBlockFPs.length - lastLevel);
+ System.arraycopy(leafBlockFPs, 0, newLeafBlockFPs, leafBlockFPs.length - lastLevel, lastLevel);
+ leafBlockFPs = newLeafBlockFPs;
+ }
+ break;
+ }
+
+ levelCount *= 2;
+ }
+ }
+
+ // Reused while packing the index
+ try (ResettableByteBuffersIndexOutput writeBuffer = new ResettableByteBuffersIndexOutput("PackedIndex"))
+ {
+ // This is the "file" we append the byte[] to:
+ List<byte[]> blocks = new ArrayList<>();
+ byte[] lastSplitValue = new byte[bytesPerValue];
+ int totalSize = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, 0, blocks, 1, lastSplitValue, false);
+ // Compact the byte[] blocks into single byte index:
+ byte[] index = new byte[totalSize];
+ int upto = 0;
+ for (byte[] block : blocks)
+ {
+ System.arraycopy(block, 0, index, upto, block.length);
+ upto += block.length;
+ }
+ assert upto == totalSize;
+
+ return index;
+ }
+ }
+
+ /**
+ * lastSplitValue is the split value previously seen; we use this to prefix-code the split byte[] on each
+ * inner node
+ */
+ private int recursePackIndex(ResettableByteBuffersIndexOutput writeBuffer, long[] leafBlockFPs, byte[] splitValues,
+ long minBlockFP, List<byte[]> blocks, int nodeID, byte[] lastSplitValue, boolean isLeft) throws IOException
+ {
+ if (nodeID >= leafBlockFPs.length)
+ {
+ int leafID = nodeID - leafBlockFPs.length;
+
+ // In the unbalanced case it's possible the left most node only has one child:
+ if (leafID < leafBlockFPs.length)
+ {
+ long delta = leafBlockFPs[leafID] - minBlockFP;
+ if (isLeft)
+ {
+ assert delta == 0;
+ return 0;
+ }
+ else
+ {
+ assert nodeID == 1 || delta > 0 : "nodeID=" + nodeID;
+ writeBuffer.writeVLong(delta);
+ return appendBlock(writeBuffer, blocks);
+ }
+ }
+ else
+ {
+ throw new IllegalStateException("Unbalanced tree");
+ }
+ }
+ else
+ {
+ long leftBlockFP;
+ if (!isLeft)
+ {
+ leftBlockFP = getLeftMostLeafBlockFP(leafBlockFPs, nodeID);
+ long delta = leftBlockFP - minBlockFP;
+ assert nodeID == 1 || delta > 0;
+ writeBuffer.writeVLong(delta);
+ }
+ else
+ {
+ // The left tree's left most leaf block FP is always the minimal FP:
+ leftBlockFP = minBlockFP;
+ }
+
+ int address = nodeID * bytesPerValue;
+
+ // find common prefix with last split value in this dim:
+ int prefix = 0;
+ for (; prefix < bytesPerValue; prefix++)
+ {
+ if (splitValues[address + prefix] != lastSplitValue[prefix])
+ {
+ break;
+ }
+ }
+
+ int firstDiffByteDelta;
+ if (prefix < bytesPerValue)
+ {
+ firstDiffByteDelta = (splitValues[address + prefix] & 0xFF) - (lastSplitValue[prefix] & 0xFF);
+ // If this is left then we need to negate the delta
+ if (isLeft)
+ firstDiffByteDelta = -firstDiffByteDelta;
+ assert firstDiffByteDelta > 0;
+ }
+ else
+ {
+ firstDiffByteDelta = 0;
+ }
+
+ // pack the prefix and delta first diff byte into a single vInt:
+ int code = (firstDiffByteDelta * (1 + bytesPerValue) + prefix);
+
+ writeBuffer.writeVInt(code);
+
+ // write the split value, prefix coded vs. our parent's split value:
+ int suffix = bytesPerValue - prefix;
+ byte[] savSplitValue = new byte[suffix];
+ if (suffix > 1)
+ {
+ writeBuffer.writeBytes(splitValues, address + prefix + 1, suffix - 1);
+ }
+
+ byte[] cmp = lastSplitValue.clone();
+
+ System.arraycopy(lastSplitValue, prefix, savSplitValue, 0, suffix);
+
+ // copy our split value into lastSplitValue for our children to prefix-code against
+ System.arraycopy(splitValues, address + prefix, lastSplitValue, prefix, suffix);
+
+ int numBytes = appendBlock(writeBuffer, blocks);
+
+ // placeholder for left-tree numBytes; we need this so that at search time if we only need to recurse into
+ // the right subtree we can quickly seek to its starting point
+ int idxSav = blocks.size();
+ blocks.add(null);
+
+ int leftNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, leftBlockFP, blocks, 2 * nodeID, lastSplitValue, true);
+
+ if (nodeID * 2 < leafBlockFPs.length)
+ {
+ writeBuffer.writeVInt(leftNumBytes);
+ }
+ else
+ {
+ assert leftNumBytes == 0 : "leftNumBytes=" + leftNumBytes;
+ }
+ int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
+ byte[] bytes2 = writeBuffer.toArrayCopy();
+ writeBuffer.reset();
+ // replace our placeholder:
+ blocks.set(idxSav, bytes2);
+
+ int rightNumBytes = recursePackIndex(writeBuffer, leafBlockFPs, splitValues, leftBlockFP, blocks, 2 * nodeID + 1, lastSplitValue, false);
+
+ // restore lastSplitValue to what caller originally passed us:
+ System.arraycopy(savSplitValue, 0, lastSplitValue, prefix, suffix);
+
+ assert Arrays.equals(lastSplitValue, cmp);
+
+ return numBytes + numBytes2 + leftNumBytes + rightNumBytes;
+ }
+ }
+
+ /** Appends the current contents of writeBuffer as another block on the growing in-memory file */
+ private int appendBlock(ResettableByteBuffersIndexOutput writeBuffer, List<byte[]> blocks)
+ {
+ int pos = Math.toIntExact(writeBuffer.getFilePointer());
+ byte[] bytes = writeBuffer.toArrayCopy();
+ writeBuffer.reset();
+ blocks.add(bytes);
+ return pos;
+ }
+
+ private long getLeftMostLeafBlockFP(long[] leafBlockFPs, int nodeID)
+ {
+ // TODO: can we do this cheaper, e.g. a closed form solution instead of while loop? Or
+ // change the recursion while packing the index to return this left-most leaf block FP
+ // from each recursion instead?
+ //
+ // Still, the overall cost here is minor: this method's cost is O(log(N)), and while writing
+ // we call it O(N) times (N = number of leaf blocks)
+ while (nodeID < leafBlockFPs.length)
+ {
+ nodeID *= 2;
+ }
+ int leafID = nodeID - leafBlockFPs.length;
+ long result = leafBlockFPs[leafID];
+ if (result < 0)
+ {
+ throw new AssertionError(result + " for leaf " + leafID);
+ }
+ return result;
+ }
+
+ interface Callback
+ {
+ void writeLeafPostings(RowIDAndIndex[] leafPostings, int offset, int count);
+ }
+
+ static class RowIDAndIndex
+ {
+ public int valueOrderIndex;
+ public long rowID;
+
+ @Override
+ public String toString()
+ {
+ return MoreObjects.toStringHelper(this)
+ .add("valueOrderIndex", valueOrderIndex)
+ .add("rowID", rowID)
+ .toString();
+ }
+ }
+
+ /**
+ * Responsible for writing the leaf blocks at the beginning of the balanced tree index.
+ */
+ private class LeafWriter
+ {
+ private final IndexOutput treeOutput;
+ private final List<Long> leafBlockFilePointers = new ArrayList<>();
+ private final List<byte[]> leafBlockStartValues = new ArrayList<>();
+ private final byte[] leafValues = new byte[maxPointsInLeafNode * bytesPerValue];
+ private final long[] leafRowIDs = new long[maxPointsInLeafNode];
+ private final RowIDAndIndex[] rowIDAndIndexes = new RowIDAndIndex[maxPointsInLeafNode];
+ private final int[] orderIndex = new int[maxPointsInLeafNode];
+ private final Callback callback;
+ private final ByteBuffersDataOutput leafOrderIndexOutput = new ByteBuffersDataOutput(2 * 1024);
+ private final ByteBuffersDataOutput leafBlockOutput = new ByteBuffersDataOutput(32 * 1024);
+ private final byte[] lastPackedValue = new byte[bytesPerValue];
+
+ private long valueCount;
+ private int leafValueCount;
+ private long lastRowID;
+
+ LeafWriter(IndexOutput treeOutput, Callback callback)
+ {
+ assert callback != null : "Callback cannot be null in TreeWriter";
+
+ this.treeOutput = treeOutput;
+ this.callback = callback;
+
+ for (int x = 0; x < rowIDAndIndexes.length; x++)
+ {
+ rowIDAndIndexes[x] = new RowIDAndIndex();
+ }
+ }
+
+ /**
+ * Adds a value and row ID to the current leaf block. If the leaf block is full after the addition
+ * the current leaf block is written to disk.
+ */
+ void add(byte[] packedValue, long rowID) throws IOException
+ {
+ if (DEBUG)
+ valueInOrder(valueCount + leafValueCount, lastPackedValue, packedValue, 0, rowID, lastRowID);
+
+ System.arraycopy(packedValue, 0, leafValues, leafValueCount * bytesPerValue, bytesPerValue);
+ leafRowIDs[leafValueCount] = rowID;
+ leafValueCount++;
+
+ if (leafValueCount == maxPointsInLeafNode)
+ {
+ // We write a block once we hit exactly the max count
+ writeLeafBlock();
+ leafValueCount = 0;
+ }
+
+ if (DEBUG)
+ if ((lastRowID = rowID) < 0)
+ throw new AssertionError("row id must be >= 0; got " + rowID);
+ }
+
+ /**
+ * Write a leaf block if we have unwritten values and return the total number of values added
+ */
+ public long finish() throws IOException
+ {
+ if (leafValueCount > 0)
+ writeLeafBlock();
+
+ return valueCount;
+ }
+
+ private void writeLeafBlock() throws IOException
+ {
+ assert leafValueCount != 0;
+ if (valueCount == 0)
+ {
+ System.arraycopy(leafValues, 0, minPackedValue, 0, bytesPerValue);
+ }
+ System.arraycopy(leafValues, (leafValueCount - 1) * bytesPerValue, maxPackedValue, 0, bytesPerValue);
+
+ valueCount += leafValueCount;
+
+ if (leafBlockFilePointers.size() > 0)
+ {
+ // Save the first (minimum) value in each leaf block except the first, to build the split value index in the end:
+ leafBlockStartValues.add(ArrayUtil.copyOfSubArray(leafValues, 0, bytesPerValue));
+ }
+ leafBlockFilePointers.add(treeOutput.getFilePointer());
+ checkMaxLeafNodeCount(leafBlockFilePointers.size());
+
+ // Find the common prefix between the first and last values in the block
+ int commonPrefixLength = bytesPerValue;
+ int offset = (leafValueCount - 1) * bytesPerValue;
+ for (int j = 0; j < bytesPerValue; j++)
+ {
+ if (leafValues[j] != leafValues[offset + j])
+ {
+ commonPrefixLength = j;
+ break;
+ }
+ }
+
+ treeOutput.writeVInt(leafValueCount);
+
+ for (int x = 0; x < leafValueCount; x++)
+ {
+ rowIDAndIndexes[x].valueOrderIndex = x;
+ rowIDAndIndexes[x].rowID = leafRowIDs[x];
+ }
+
+ final Sorter sorter = new IntroSorter()
+ {
+ RowIDAndIndex pivot;
+
+ @Override
+ protected void swap(int i, int j)
+ {
+ RowIDAndIndex o = rowIDAndIndexes[i];
+ rowIDAndIndexes[i] = rowIDAndIndexes[j];
+ rowIDAndIndexes[j] = o;
+ }
+
+ @Override
+ protected void setPivot(int i)
+ {
+ pivot = rowIDAndIndexes[i];
+ }
+
+ @Override
+ protected int comparePivot(int j)
+ {
+ return Long.compare(pivot.rowID, rowIDAndIndexes[j].rowID);
+ }
+ };
+
+ sorter.sort(0, leafValueCount);
+
+ // write the leaf order index: leaf rowID -> orig index
+ leafOrderIndexOutput.reset();
+
+ // iterate in row ID order to get the row ID index for the given value order index
+ // place into an array to be written as packed ints
+ for (int x = 0; x < leafValueCount; x++)
+ orderIndex[rowIDAndIndexes[x].valueOrderIndex] = x;
+
+ LeafOrderMap.write(orderIndex, leafValueCount, maxPointsInLeafNode - 1, leafOrderIndexOutput);
+
+ treeOutput.writeVInt((int) leafOrderIndexOutput.size());
+ leafOrderIndexOutput.copyTo(treeOutput);
+
+ callback.writeLeafPostings(rowIDAndIndexes, 0, leafValueCount);
+
+ // Write the common prefix for the leaf block
+ writeCommonPrefix(treeOutput, commonPrefixLength, leafValues);
+
+ // Write the run length encoded packed values for the leaf block
+ leafBlockOutput.reset();
+
+ if (DEBUG)
+ valuesInOrderAndBounds(leafValueCount,
+ ArrayUtil.copyOfSubArray(leafValues, 0, bytesPerValue),
+ ArrayUtil.copyOfSubArray(leafValues, (leafValueCount - 1) * bytesPerValue, leafValueCount * bytesPerValue),
+ leafRowIDs);
+
+ writeLeafBlockPackedValues(leafBlockOutput, commonPrefixLength, leafValueCount);
+
+ leafBlockOutput.copyTo(treeOutput);
+ }
+
+ private void checkMaxLeafNodeCount(int numLeaves)
+ {
+ if (bytesPerValue * (long) numLeaves > ArrayUtil.MAX_ARRAY_LENGTH)
+ {
+ throw new IllegalStateException("too many nodes; increase maxPointsInLeafNode (currently " + maxPointsInLeafNode + ") and reindex");
+ }
+ }
+
+ private void writeCommonPrefix(DataOutput treeOutput, int commonPrefixLength, byte[] packedValue) throws IOException
+ {
+ treeOutput.writeVInt(commonPrefixLength);
+ if (commonPrefixLength > 0)
+ treeOutput.writeBytes(packedValue, 0, commonPrefixLength);
+ }
+
+ private void writeLeafBlockPackedValues(DataOutput out, int commonPrefixLength, int count) throws IOException
+ {
+ // If all the values are the same (e.g. the common prefix length == bytes per value) then we don't
+ // need to write anything. Otherwise, we run length compress the values to disk.
+ if (commonPrefixLength != bytesPerValue)
+ {
+ int compressedByteOffset = commonPrefixLength;
+ commonPrefixLength++;
+ for (int i = 0; i < count; )
+ {
+ // do run-length compression on the byte at compressedByteOffset
+ int runLen = runLen(i, Math.min(i + 0xff, count), compressedByteOffset);
+ assert runLen <= 0xff;
+ byte prefixByte = leafValues[i * bytesPerValue + compressedByteOffset];
+ out.writeByte(prefixByte);
+ out.writeByte((byte) runLen);
+ writeLeafBlockPackedValuesRange(out, commonPrefixLength, i, i + runLen);
+ i += runLen;
+ assert i <= count;
+ }
+ }
+ }
+
+ private void writeLeafBlockPackedValuesRange(DataOutput out, int commonPrefixLength, int start, int end) throws IOException
+ {
+ for (int i = start; i < end; ++i)
+ {
+ out.writeBytes(leafValues, i * bytesPerValue + commonPrefixLength, bytesPerValue - commonPrefixLength);
+ }
+ }
+
+ private int runLen(int start, int end, int byteOffset)
+ {
+ byte b = leafValues[start * bytesPerValue + byteOffset];
+ for (int i = start + 1; i < end; ++i)
+ {
+ byte b2 = leafValues[i * bytesPerValue + byteOffset];
+ assert Byte.toUnsignedInt(b2) >= Byte.toUnsignedInt(b);
+ if (b != b2)
+ {
+ return i - start;
+ }
+ }
+ return end - start;
+ }
+
+ // The following 3 methods are only used when DEBUG is true:
+
+ private void valueInBounds(byte[] packedValue, int packedValueOffset, byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ if (ByteArrayUtil.compareUnsigned(packedValue,
+ packedValueOffset,
+ minPackedValue,
+ 0,
+ bytesPerValue) < 0)
+ {
+ throw new AssertionError("value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+ " is < minPackedValue=" + new BytesRef(minPackedValue));
+ }
+
+ if (ByteArrayUtil.compareUnsigned(packedValue,
+ packedValueOffset,
+ maxPackedValue, 0,
+ bytesPerValue) > 0)
+ {
+ throw new AssertionError("value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+ " is > maxPackedValue=" + new BytesRef(maxPackedValue));
+ }
+ }
+
+ private void valuesInOrderAndBounds(int count, byte[] minPackedValue, byte[] maxPackedValue, long[] rowIds)
+ {
+ byte[] lastPackedValue = new byte[bytesPerValue];
+ long lastRowId = -1;
+ for (int i = 0; i < count; i++)
+ {
+ valueInOrder(i, lastPackedValue, leafValues, i * bytesPerValue, rowIds[i], lastRowId);
+ lastRowId = rowIds[i];
+
+ // Make sure this value does in fact fall within this leaf cell:
+ valueInBounds(leafValues, i * bytesPerValue, minPackedValue, maxPackedValue);
+ }
+ }
+
+ private void valueInOrder(long ord, byte[] lastPackedValue, byte[] packedValue, int packedValueOffset, long rowId, long lastRowId)
+ {
+ if (ord > 0)
+ {
+ int cmp = ByteArrayUtil.compareUnsigned(lastPackedValue, 0, packedValue, packedValueOffset, bytesPerValue);
+ if (cmp > 0)
+ {
+ throw new AssertionError("values out of order: last value=" + new BytesRef(lastPackedValue) +
+ " current value=" + new BytesRef(packedValue, packedValueOffset, bytesPerValue) +
+ " ord=" + ord);
+ }
+ if (cmp == 0 && rowId < lastRowId)
+ {
+ throw new AssertionError("row IDs out of order: last rowID=" + lastRowId + " current rowID=" + rowId + " ord=" + ord);
+ }
+ }
+ System.arraycopy(packedValue, packedValueOffset, lastPackedValue, 0, bytesPerValue);
+ }
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java
new file mode 100644
index 0000000..8fd5bf3
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMap.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.packed.DirectWriter;
+
+class LeafOrderMap
+{
+ static void write(final int[] array, int length, int maxValue, final DataOutput out) throws IOException
+ {
+ final int bits = DirectWriter.unsignedBitsRequired(maxValue);
+ final DirectWriter writer = DirectWriter.getInstance(out, length, bits);
+ for (int i = 0; i < length; i++)
+ {
+ assert array[i] <= maxValue;
+
+ writer.add(array[i]);
+ }
+ writer.finish();
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java
new file mode 100644
index 0000000..3f02d5d
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriter.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.MoreObjects;
+
+import org.apache.cassandra.index.sai.IndexContext;
+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.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Specialized writer for values, that builds them into a {@link BlockBalancedTreeWriter} with auxiliary
+ * posting lists on eligible tree levels.
+ * <p>
+ * Given a sorted input {@link BlockBalancedTreeIterator}, the flush process is optimised because we don't need to
+ * buffer all point values to sort them.
+ */
+public class NumericIndexWriter
+{
+ public static final int MAX_POINTS_IN_LEAF_NODE = BlockBalancedTreeWriter.DEFAULT_MAX_POINTS_IN_LEAF_NODE;
+ private static final int DEFAULT_POSTINGS_SIZE = 128;
+
+ private final BlockBalancedTreeWriter writer;
+ private final IndexDescriptor indexDescriptor;
+ private final IndexContext indexContext;
+ private final int bytesPerValue;
+
+ /**
+ * @param maxSegmentRowId maximum possible segment row ID, used to create `maxRows` for the balanced tree
+ */
+ public NumericIndexWriter(IndexDescriptor indexDescriptor,
+ IndexContext indexContext,
+ int bytesPerValue,
+ long maxSegmentRowId)
+ {
+ this(indexDescriptor, indexContext, MAX_POINTS_IN_LEAF_NODE, bytesPerValue, maxSegmentRowId);
+ }
+
+ @VisibleForTesting
+ public NumericIndexWriter(IndexDescriptor indexDescriptor,
+ IndexContext indexContext,
+ int maxPointsInLeafNode,
+ int bytesPerValue,
+ long maxSegmentRowId)
+ {
+ checkArgument(maxSegmentRowId >= 0, "[%s] maxSegmentRowId must be non-negative value, but got %s", indexContext.getIndexName(), maxSegmentRowId);
+
+ this.indexDescriptor = indexDescriptor;
+ this.indexContext = indexContext;
+ this.bytesPerValue = bytesPerValue;
+ this.writer = new BlockBalancedTreeWriter(bytesPerValue, maxPointsInLeafNode);
+ }
+
+ @Override
+ public String toString()
+ {
+ return MoreObjects.toStringHelper(this)
+ .add("indexContext", indexContext)
+ .add("bytesPerValue", bytesPerValue)
+ .toString();
+ }
+
+ private static class LeafCallback implements BlockBalancedTreeWriter.Callback
+ {
+ final List<PackedLongValues> leafPostings = new ArrayList<>(DEFAULT_POSTINGS_SIZE);
+
+ public int numLeaves()
+ {
+ return leafPostings.size();
+ }
+
+ @Override
+ public void writeLeafPostings(BlockBalancedTreeWriter.RowIDAndIndex[] leafPostings, int offset, int count)
+ {
+ PackedLongValues.Builder builder = PackedLongValues.monotonicBuilder(PackedInts.COMPACT);
+
+ for (int i = offset; i < count; ++i)
+ {
+ builder.add(leafPostings[i].rowID);
+ }
+ this.leafPostings.add(builder.build());
+ }
+ }
+
+ /**
+ * Writes a balanced tree and posting lists from a {@link BlockBalancedTreeIterator}.
+ *
+ * @param values sorted {@link BlockBalancedTreeIterator} values to write
+ *
+ * @return metadata describing the location and size of this balanced tree in the overall SSTable balanced tree component file
+ */
+ public SegmentMetadata.ComponentMetadataMap writeCompleteSegment(BlockBalancedTreeIterator values) throws IOException
+ {
+ long treePosition;
+
+ SegmentMetadata.ComponentMetadataMap components = new SegmentMetadata.ComponentMetadataMap();
+
+ LeafCallback leafCallback = new LeafCallback();
+
+ try (IndexOutput treeOutput = indexDescriptor.openPerIndexOutput(IndexComponent.BALANCED_TREE, indexContext, true))
+ {
+ // The SSTable balanced tree component file is opened in append mode, so our offset is the current file pointer.
+ long treeOffset = treeOutput.getFilePointer();
+
+ treePosition = writer.write(treeOutput, values, leafCallback);
+
+ // If the treePosition is less than 0 then we didn't write any values out and the index is empty
+ if (treePosition < 0)
+ return components;
+
+ long treeLength = treeOutput.getFilePointer() - treeOffset;
+
+ Map<String, String> attributes = new LinkedHashMap<>();
+ attributes.put("max_points_in_leaf_node", Integer.toString(writer.getMaxPointsInLeafNode()));
+ attributes.put("num_leaves", Integer.toString(leafCallback.numLeaves()));
+ attributes.put("num_values", Long.toString(writer.getValueCount()));
+ attributes.put("bytes_per_value", Long.toString(writer.getBytesPerValue()));
+
+ components.put(IndexComponent.BALANCED_TREE, treePosition, treeOffset, treeLength, attributes);
+ }
+
+ try (BlockBalancedTreeWalker reader = new BlockBalancedTreeWalker(indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext), treePosition);
+ IndexOutputWriter postingsOutput = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext, true))
+ {
+ long postingsOffset = postingsOutput.getFilePointer();
+
+ BlockBalancedTreePostingsWriter postingsWriter = new BlockBalancedTreePostingsWriter();
+ reader.traverse(postingsWriter);
+
+ // The balanced tree postings writer already writes its own header & footer.
+ long postingsPosition = postingsWriter.finish(postingsOutput, leafCallback.leafPostings, indexContext);
+
+ Map<String, String> attributes = new LinkedHashMap<>();
+ attributes.put("num_leaf_postings", Integer.toString(postingsWriter.numLeafPostings));
+ attributes.put("num_non_leaf_postings", Integer.toString(postingsWriter.numNonLeafPostings));
+
+ long postingsLength = postingsOutput.getFilePointer() - postingsOffset;
+ components.put(IndexComponent.POSTING_LISTS, postingsPosition, postingsOffset, postingsLength, attributes);
+ }
+
+ return components;
+ }
+
+ /**
+ * @return number of values added
+ */
+ public long getValueCount()
+ {
+ return writer.getValueCount();
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java
new file mode 100644
index 0000000..9140e35
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/FilteringPostingList.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.postings;
+
+import java.io.IOException;
+
+import org.apache.cassandra.index.sai.postings.OrdinalPostingList;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * A wrapper that iterates over a delegate {@link PostingList}, filtering out postings at
+ * positions that are not present in a provided filter.
+ */
+public class FilteringPostingList implements PostingList
+{
+ private final FixedBitSet filter;
+ private final OrdinalPostingList delegate;
+ private final int cardinality;
+ private int position = 0;
+
+ public FilteringPostingList(FixedBitSet filter, OrdinalPostingList delegate)
+ {
+ cardinality = filter.cardinality();
+ this.filter = filter;
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void close()
+ {
+ delegate.close();
+ }
+
+ /**
+ *
+ * @return the segment row ID of the next match
+ */
+ @Override
+ public long nextPosting() throws IOException
+ {
+ while (true)
+ {
+ long segmentRowId = delegate.nextPosting();
+
+ if (segmentRowId == PostingList.END_OF_STREAM)
+ {
+ return PostingList.END_OF_STREAM;
+ }
+
+ if (filter.get(position++))
+ {
+ return segmentRowId;
+ }
+ }
+ }
+
+ @Override
+ public long size()
+ {
+ return cardinality;
+ }
+
+ @Override
+ public long advance(long targetRowID) throws IOException
+ {
+ long segmentRowId = delegate.advance(targetRowID);
+
+ if (segmentRowId == PostingList.END_OF_STREAM)
+ {
+ return PostingList.END_OF_STREAM;
+ }
+
+ // these are always for leaf balanced tree postings so the max is 1024
+ position = (int)delegate.getOrdinal();
+
+ // If the ordinal of the ID we just read satisfies the filter, just return it...
+ if (filter.get(position - 1))
+ {
+ return segmentRowId;
+ }
+
+ // ...but if the ID doesn't satisfy the filter, get the next match.
+ return nextPosting();
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.java
new file mode 100644
index 0000000..34a6ea8
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PackedLongsPostingList.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.postings;
+
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+/**
+ * Adapter class for {@link PackedLongValues} to expose it as {@link PostingList}.
+ */
+public class PackedLongsPostingList implements PostingList
+{
+ private final PackedLongValues.Iterator iterator;
+ private final PackedLongValues values;
+
+ public PackedLongsPostingList(PackedLongValues values)
+ {
+ this.values = values;
+ iterator = values.iterator();
+ }
+
+ @Override
+ public long nextPosting()
+ {
+ if (iterator.hasNext())
+ {
+ return iterator.next();
+ }
+ else
+ {
+ return PostingList.END_OF_STREAM;
+ }
+ }
+
+ @Override
+ public long size()
+ {
+ return values.size();
+ }
+
+ @Override
+ public long advance(long targetRowID)
+ {
+ throw new UnsupportedOperationException();
+ }
+}
+
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
index 608d837..0a413a5 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsReader.java
@@ -74,7 +74,6 @@
public PostingsReader(IndexInput input, BlocksSummary summary, QueryEventListener.PostingListEventListener listener) throws IOException
{
- logger.debug("Opening postings reader for {}", input);
this.input = input;
this.seekingInput = new SeekingRandomAccessInput(input);
this.listener = listener;
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
index 8f12532..8e428d5 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/postings/PostingsWriter.java
@@ -43,7 +43,7 @@
/**
* Encodes, compresses and writes postings lists to disk.
- *
+ * <p>
* All postings in the posting list are delta encoded, then deltas are divided into blocks for compression.
* The deltas are based on the final value of the previous block. For the first block in the posting list
* the first value in the block is written as a VLong prior to block delta encodings.
@@ -114,16 +114,19 @@
this(indexDescriptor, indexContext, BLOCK_SIZE);
}
+ public PostingsWriter(IndexOutputWriter dataOutput) throws IOException
+ {
+ this(dataOutput, BLOCK_SIZE);
+ }
+
@VisibleForTesting
PostingsWriter(IndexDescriptor indexDescriptor, IndexContext indexContext, int blockSize) throws IOException
{
this(indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext, true), blockSize);
}
- private PostingsWriter(IndexOutput dataOutput, int blockSize) throws IOException
+ private PostingsWriter(IndexOutputWriter dataOutput, int blockSize) throws IOException
{
- assert dataOutput instanceof IndexOutputWriter;
- logger.debug("Creating postings writer for output {}", dataOutput);
this.blockSize = blockSize;
this.dataOutput = dataOutput;
startOffset = dataOutput.getFilePointer();
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
index bb22ff4..b5e8958 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/IndexSegmentSearcher.java
@@ -54,12 +54,15 @@
this.indexContext = indexContext;
}
+ @SuppressWarnings({"resource", "RedundantSuppression"})
public static IndexSegmentSearcher open(PrimaryKeyMap.Factory primaryKeyMapFactory,
PerColumnIndexFiles indexFiles,
SegmentMetadata segmentMetadata,
IndexContext indexContext) throws IOException
{
- return new LiteralIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext);
+ return indexContext.isLiteral()
+ ? new LiteralIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext)
+ : new NumericIndexSegmentSearcher(primaryKeyMapFactory, indexFiles, segmentMetadata, indexContext);
}
/**
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java
new file mode 100644
index 0000000..9f50695
--- /dev/null
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/NumericIndexSegmentSearcher.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.index.sai.disk.v1.segment;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+import com.google.common.base.MoreObjects;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.disk.PrimaryKeyMap;
+import org.apache.cassandra.index.sai.disk.format.IndexComponent;
+import org.apache.cassandra.index.sai.disk.v1.PerColumnIndexFiles;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeReader;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.metrics.MulticastQueryEventListeners;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.lucene.index.CorruptIndexException;
+
+import static org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeQueries.balancedTreeQueryFrom;
+
+/**
+ * Executes {@link Expression}s against the balanced tree for an individual index segment.
+ */
+public class NumericIndexSegmentSearcher extends IndexSegmentSearcher
+{
+ private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+ private final BlockBalancedTreeReader treeReader;
+ private final QueryEventListener.BalancedTreeEventListener perColumnEventListener;
+
+ NumericIndexSegmentSearcher(PrimaryKeyMap.Factory primaryKeyMapFactory,
+ PerColumnIndexFiles perIndexFiles,
+ SegmentMetadata segmentMetadata,
+ IndexContext indexContext) throws IOException
+ {
+ super(primaryKeyMapFactory, perIndexFiles, segmentMetadata, indexContext);
+
+ final long treePosition = metadata.getIndexRoot(IndexComponent.BALANCED_TREE);
+ if (treePosition < 0)
+ throw new CorruptIndexException(indexContext.logMessage("The tree position is less than zero."), IndexComponent.BALANCED_TREE.name);
+ final long postingsPosition = metadata.getIndexRoot(IndexComponent.POSTING_LISTS);
+ if (postingsPosition < 0)
+ throw new CorruptIndexException(indexContext.logMessage("The postings position is less than zero."), IndexComponent.BALANCED_TREE.name);
+
+ treeReader = new BlockBalancedTreeReader(indexContext,
+ indexFiles.balancedTree(),
+ treePosition,
+ indexFiles.postingLists(),
+ postingsPosition);
+ perColumnEventListener = (QueryEventListener.BalancedTreeEventListener)indexContext.getColumnQueryMetrics();
+ }
+
+ @Override
+ public long indexFileCacheSize()
+ {
+ return treeReader.memoryUsage();
+ }
+
+ @Override
+ @SuppressWarnings({"resource", "RedundantSuppression"})
+ public KeyRangeIterator search(Expression exp, QueryContext context) throws IOException
+ {
+ if (logger.isTraceEnabled())
+ logger.trace(indexContext.logMessage("Searching on expression '{}'..."), exp);
+
+ if (exp.getOp().isEqualityOrRange())
+ {
+ final BlockBalancedTreeReader.IntersectVisitor query = balancedTreeQueryFrom(exp, treeReader.getBytesPerValue());
+ QueryEventListener.BalancedTreeEventListener listener = MulticastQueryEventListeners.of(context, perColumnEventListener);
+ PostingList postingList = treeReader.intersect(query, listener, context);
+ return toIterator(postingList, context);
+ }
+ else
+ {
+ throw new IllegalArgumentException(indexContext.logMessage("Unsupported expression during index query: " + exp));
+ }
+ }
+
+ @Override
+ public String toString()
+ {
+ return MoreObjects.toStringHelper(this)
+ .add("indexContext", indexContext)
+ .add("count", treeReader.getPointCount())
+ .add("bytesPerValue", treeReader.getBytesPerValue())
+ .toString();
+ }
+
+ @Override
+ public void close()
+ {
+ treeReader.close();
+ }
+}
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
index 5f108a6..8c84991 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/Segment.java
@@ -36,7 +36,7 @@
import org.apache.cassandra.io.util.FileUtils;
/**
- * Each segment represents an on-disk index structure (kdtree/terms/postings) flushed by memory limit or token boundaries.
+ * Each segment represents an on-disk index structure (balanced tree/terms/postings) flushed by memory limit or token boundaries.
* It also helps to reduce resource consumption for read requests as only segments that intersect with read request data
* range need to be loaded.
*/
diff --git a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
index dd86662..44ee9fc 100644
--- a/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
+++ b/src/java/org/apache/cassandra/index/sai/disk/v1/segment/SegmentBuilder.java
@@ -28,9 +28,11 @@
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.index.sai.IndexContext;
-import org.apache.cassandra.index.sai.memory.RAMStringIndexer;
import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeRamBuffer;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
import org.apache.cassandra.index.sai.disk.v1.trie.LiteralIndexWriter;
+import org.apache.cassandra.index.sai.memory.RAMStringIndexer;
import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter;
import org.apache.cassandra.index.sai.utils.PrimaryKey;
import org.apache.cassandra.index.sai.utils.TypeUtil;
@@ -54,7 +56,6 @@
/** Minimum flush size, dynamically updated as segment builds are started and completed/aborted. */
private static volatile long minimumFlushBytes;
- private final AbstractType<?> termComparator;
private final NamedMemoryLimiter limiter;
private final long lastValidSegmentRowID;
private boolean flushed = false;
@@ -71,10 +72,49 @@
private ByteBuffer minTerm;
private ByteBuffer maxTerm;
+ final AbstractType<?> termComparator;
long totalBytesAllocated;
int rowCount = 0;
int maxSegmentRowId = -1;
+ public static class BlockBalancedTreeSegmentBuilder extends SegmentBuilder
+ {
+ private final byte[] scratch;
+ private final BlockBalancedTreeRamBuffer trieBuffer;
+
+ public BlockBalancedTreeSegmentBuilder(AbstractType<?> termComparator, NamedMemoryLimiter limiter)
+ {
+ super(termComparator, limiter);
+
+ scratch = new byte[TypeUtil.fixedSizeOf(termComparator)];
+ trieBuffer = new BlockBalancedTreeRamBuffer(TypeUtil.fixedSizeOf(termComparator));
+ totalBytesAllocated = this.trieBuffer.memoryUsed();
+ }
+
+ @Override
+ public boolean isEmpty()
+ {
+ return trieBuffer.numRows() == 0;
+ }
+
+ @Override
+ protected long addInternal(ByteBuffer term, int segmentRowId)
+ {
+ TypeUtil.toComparableBytes(term, termComparator, scratch);
+ return trieBuffer.add(segmentRowId, scratch);
+ }
+
+ @Override
+ protected SegmentMetadata.ComponentMetadataMap flushInternal(IndexDescriptor indexDescriptor, IndexContext indexContext) throws IOException
+ {
+ NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ indexContext,
+ TypeUtil.fixedSizeOf(termComparator),
+ maxSegmentRowId);
+ return writer.writeCompleteSegment(trieBuffer.iterator());
+ }
+ }
+
public static class RAMStringSegmentBuilder extends SegmentBuilder
{
final RAMStringIndexer ramIndexer;
@@ -89,11 +129,13 @@
totalBytesAllocated = ramIndexer.estimatedBytesUsed();
}
+ @Override
public boolean isEmpty()
{
return ramIndexer.isEmpty();
}
+ @Override
protected long addInternal(ByteBuffer term, int segmentRowId)
{
copyBufferToBytesRef(term, stringBuffer);
@@ -128,7 +170,7 @@
{
this.termComparator = termComparator;
this.limiter = limiter;
- this.lastValidSegmentRowID = testLastValidSegmentRowId >= 0 ? testLastValidSegmentRowId : LAST_VALID_SEGMENT_ROW_ID;
+ lastValidSegmentRowID = testLastValidSegmentRowId >= 0 ? testLastValidSegmentRowId : LAST_VALID_SEGMENT_ROW_ID;
minimumFlushBytes = limiter.limitBytes() / ACTIVE_BUILDER_COUNT.incrementAndGet();
}
@@ -204,7 +246,7 @@
/**
* This method does three things:
- *
+ * <p>
* 1. It decrements active builder count and updates the global minimum flush size to reflect that.
* 2. It releases the builder's memory against its limiter.
* 3. It defensively marks the builder inactive to make sure nothing bad happens if we try to close it twice.
diff --git a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
index 89b8f7f..f5bca79 100644
--- a/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
+++ b/src/java/org/apache/cassandra/index/sai/iterators/KeyRangeIntersectionIterator.java
@@ -41,13 +41,9 @@
{
private static final Logger logger = LoggerFactory.getLogger(KeyRangeIntersectionIterator.class);
- // The cassandra.sai.intersection.clause.limit (default: 2) controls the maximum number of range iterator that
- // will be used in the final intersection of a query operation.
- private static final int INTERSECTION_CLAUSE_LIMIT = CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt();
-
static
{
- logger.info(String.format("Storage attached index intersection clause limit is %d", INTERSECTION_CLAUSE_LIMIT));
+ logger.info(String.format("Storage attached index intersection clause limit is %d", CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt()));
}
private final List<KeyRangeIterator> ranges;
@@ -133,7 +129,7 @@
public static Builder builder(int size)
{
- return builder(size, INTERSECTION_CLAUSE_LIMIT);
+ return new Builder(size);
}
@VisibleForTesting
@@ -145,6 +141,9 @@
@VisibleForTesting
public static class Builder extends KeyRangeIterator.Builder
{
+ // This controls the maximum number of range iterators that will be used in the final
+ // intersection of a query operation. It is set from cassandra.sai.intersection_clause_limit
+ // and defaults to 2
private final int limit;
// tracks if any of the added ranges are disjoint with the other ranges, which is useful
// in case of intersection, as it gives a direct answer whether the iterator is going
@@ -153,6 +152,11 @@
protected final List<KeyRangeIterator> rangeIterators;
+ Builder(int size)
+ {
+ this(size, CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.getInt());
+ }
+
Builder(int size, int limit)
{
super(new IntersectionStatistics());
diff --git a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
index d9554e4..e84a3f9 100644
--- a/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
+++ b/src/java/org/apache/cassandra/index/sai/memory/TrieMemoryIndex.java
@@ -218,7 +218,6 @@
{
return isLiteral ? version -> ByteSourceInverse.unescape(ByteSource.peekable(term.asComparableBytes(version)))
: term;
-
}
private ByteSource terminated(ByteSource src)
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
index 977f5f4..7259ecc 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/ColumnQueryMetrics.java
@@ -70,6 +70,61 @@
}
}
+ public static class BalancedTreeIndexMetrics extends ColumnQueryMetrics implements QueryEventListener.BalancedTreeEventListener
+ {
+ private static final String BALANCED_TREE_POSTINGS_TYPE = "BalancedTreePostings";
+
+ /**
+ * Balanced Tree index metrics.
+ */
+ private final Timer intersectionLatency;
+ private final Meter postingsNumPostings;
+ private final Meter intersectionEarlyExits;
+
+ private final QueryEventListener.PostingListEventListener postingsListener;
+
+ public BalancedTreeIndexMetrics(IndexContext indexContext)
+ {
+ super(indexContext);
+
+ intersectionLatency = Metrics.timer(createMetricName("BalancedTreeIntersectionLatency"));
+ intersectionEarlyExits = Metrics.meter(createMetricName("BalancedTreeIntersectionEarlyExits"));
+
+ postingsNumPostings = Metrics.meter(createMetricName("NumPostings", BALANCED_TREE_POSTINGS_TYPE));
+
+ Meter postingDecodes = Metrics.meter(createMetricName("PostingDecodes", BALANCED_TREE_POSTINGS_TYPE));
+
+ postingsListener = new PostingListEventsMetrics(postingDecodes);
+ }
+
+ @Override
+ public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit)
+ {
+ intersectionLatency.update(intersectionTotalTime, unit);
+ }
+
+ @Override
+ public void onIntersectionEarlyExit()
+ {
+ intersectionEarlyExits.mark();
+ }
+
+ @Override
+ public void postingListsHit(int count)
+ {
+ postingsNumPostings.mark(count);
+ }
+
+ @Override
+ public void onSegmentHit() { }
+
+ @Override
+ public QueryEventListener.PostingListEventListener postingListEventListener()
+ {
+ return postingsListener;
+ }
+ }
+
private static class PostingListEventsMetrics implements QueryEventListener.PostingListEventListener
{
private final Meter postingDecodes;
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
index f0b81cc..5495f2f 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/MulticastQueryEventListeners.java
@@ -28,6 +28,11 @@
return new Multicast2TrieIndexEventListener(ctx, listener);
}
+ public static QueryEventListener.BalancedTreeEventListener of(QueryContext ctx, QueryEventListener.BalancedTreeEventListener listener)
+ {
+ return new Multicast2BalancedTreeEventListener(ctx, listener);
+ }
+
public static class Multicast2TrieIndexEventListener implements QueryEventListener.TrieIndexEventListener
{
private final QueryContext ctx;
@@ -62,6 +67,79 @@
}
}
+ public static class Multicast2BalancedTreeEventListener implements QueryEventListener.BalancedTreeEventListener
+ {
+ private final QueryContext ctx;
+ private final QueryEventListener.BalancedTreeEventListener listener;
+ private final Multicast2BalancedTreePostingListEventListener postingListEventListener;
+
+ private Multicast2BalancedTreeEventListener(QueryContext ctx, QueryEventListener.BalancedTreeEventListener listener)
+ {
+ this.ctx = ctx;
+ this.listener = listener;
+ this.postingListEventListener = new Multicast2BalancedTreePostingListEventListener(ctx, listener.postingListEventListener());
+ }
+
+ @Override
+ public void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit)
+ {
+ listener.onIntersectionComplete(intersectionTotalTime, unit);
+ }
+
+ @Override
+ public void onIntersectionEarlyExit()
+ {
+ listener.onIntersectionEarlyExit();
+ }
+
+ @Override
+ public void postingListsHit(int count)
+ {
+ ctx.balancedTreePostingListsHit++;
+ listener.postingListsHit(count);
+ }
+
+ @Override
+ public void onSegmentHit()
+ {
+ ctx.segmentsHit++;
+ ctx.balancedTreeSegmentsHit++;
+ listener.onSegmentHit();
+ }
+
+ @Override
+ public QueryEventListener.PostingListEventListener postingListEventListener()
+ {
+ return postingListEventListener;
+ }
+ }
+
+ public static class Multicast2BalancedTreePostingListEventListener implements QueryEventListener.PostingListEventListener
+ {
+ private final QueryContext ctx;
+ private final QueryEventListener.PostingListEventListener listener;
+
+ Multicast2BalancedTreePostingListEventListener(QueryContext ctx, QueryEventListener.PostingListEventListener listener)
+ {
+ this.ctx = ctx;
+ this.listener = listener;
+ }
+
+ @Override
+ public void onAdvance()
+ {
+ ctx.balancedTreePostingsSkips++;
+ listener.onAdvance();
+ }
+
+ @Override
+ public void postingDecoded(long postingDecoded)
+ {
+ ctx.balancedTreePostingsDecodes += postingDecoded;
+ listener.postingDecoded(postingDecoded);
+ }
+ }
+
public static class Multicast2TriePostingListEventListener implements QueryEventListener.PostingListEventListener
{
private final QueryContext ctx;
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
index 7bae429..db583d8 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/QueryEventListener.java
@@ -24,6 +24,37 @@
*/
public interface QueryEventListener
{
+ /**
+ * Collector for balanced tree file related metrics.
+ */
+ interface BalancedTreeEventListener
+ {
+ /**
+ * Per-segment balanced tree index intersection time in given units. Recorded when intersection completes.
+ */
+ void onIntersectionComplete(long intersectionTotalTime, TimeUnit unit);
+
+ /**
+ * When an intersection exits early due to the query shape being completely outside the min/max range.
+ */
+ void onIntersectionEarlyExit();
+
+ /**
+ * How many balanced tree posting list were matched during the intersection.
+ */
+ void postingListsHit(int count);
+
+ /**
+ * When query potentially matches value range within a segment, and we need to do a traversal.
+ */
+ void onSegmentHit();
+
+ /**
+ * Returns events listener for balanced tree postings.
+ */
+ PostingListEventListener postingListEventListener();
+ }
+
interface TrieIndexEventListener
{
/**
@@ -62,13 +93,11 @@
@Override
public void onAdvance()
{
-
}
@Override
public void postingDecoded(long postingsDecoded)
{
-
}
};
}
diff --git a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
index 8e7b141..7154df2 100644
--- a/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
+++ b/src/java/org/apache/cassandra/index/sai/metrics/TableQueryMetrics.java
@@ -78,6 +78,16 @@
private final Histogram rowsFiltered;
/**
+ * Balanced tree index metrics.
+ */
+ private final Histogram balancedTreePostingsNumPostings;
+ /**
+ * Balanced tree index posting lists metrics.
+ */
+ private final Histogram balancedTreePostingsSkips;
+ private final Histogram balancedTreePostingsDecodes;
+
+ /**
* Trie index posting lists metrics.
*/
private final Histogram postingsSkips;
@@ -92,6 +102,11 @@
sstablesHit = Metrics.histogram(createMetricName("SSTableIndexesHit"), false);
segmentsHit = Metrics.histogram(createMetricName("IndexSegmentsHit"), false);
+ balancedTreePostingsSkips = Metrics.histogram(createMetricName("BalancedTreePostingsSkips"), false);
+
+ balancedTreePostingsNumPostings = Metrics.histogram(createMetricName("BalancedTreePostingsNumPostings"), false);
+ balancedTreePostingsDecodes = Metrics.histogram(createMetricName("BalancedTreePostingsDecodes"), false);
+
postingsSkips = Metrics.histogram(createMetricName("PostingsSkips"), false);
postingsDecodes = Metrics.histogram(createMetricName("PostingsDecodes"), false);
@@ -105,6 +120,14 @@
postingsDecodes.update(events.triePostingsDecodes);
}
+ private void recordNumericIndexCacheMetrics(QueryContext events)
+ {
+ balancedTreePostingsNumPostings.update(events.balancedTreePostingListsHit);
+
+ balancedTreePostingsSkips.update(events.balancedTreePostingsSkips);
+ balancedTreePostingsDecodes.update(events.balancedTreePostingsDecodes);
+ }
+
public void record(QueryContext queryContext)
{
final long totalQueryTimeNs = queryContext.totalQueryTimeNs();
@@ -133,6 +156,11 @@
recordStringIndexCacheMetrics(queryContext);
}
+ if (queryContext.balancedTreeSegmentsHit > 0)
+ {
+ recordNumericIndexCacheMetrics(queryContext);
+ }
+
totalQueriesCompleted.inc();
}
}
diff --git a/src/java/org/apache/cassandra/index/sai/plan/Expression.java b/src/java/org/apache/cassandra/index/sai/plan/Expression.java
index 52ebd3d..a36bf31 100644
--- a/src/java/org/apache/cassandra/index/sai/plan/Expression.java
+++ b/src/java/org/apache/cassandra/index/sai/plan/Expression.java
@@ -68,6 +68,11 @@
{
return this == EQ || this == CONTAINS_KEY || this == CONTAINS_VALUE;
}
+
+ public boolean isEqualityOrRange()
+ {
+ return isEquality() || this == RANGE;
+ }
}
public final AbstractAnalyzer.AnalyzerFactory analyzerFactory;
diff --git a/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
index 17d37f0..0a5fdf6 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/NamedMemoryLimiter.java
@@ -20,6 +20,7 @@
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.concurrent.ThreadSafe;
+import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -33,10 +34,11 @@
{
private static final Logger logger = LoggerFactory.getLogger(NamedMemoryLimiter.class);
- private final long limitBytes;
private final AtomicLong bytesUsed = new AtomicLong(0);
private final String scope;
+ private long limitBytes;
+
public NamedMemoryLimiter(long limitBytes, String scope)
{
this.limitBytes = limitBytes;
@@ -76,4 +78,10 @@
{
return limitBytes;
}
+
+ @VisibleForTesting
+ public void setLimitBytes(long bytes)
+ {
+ limitBytes = bytes;
+ }
}
diff --git a/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java b/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
index 0311e7a..6f3c1fb 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/TermsIterator.java
@@ -18,7 +18,6 @@
package org.apache.cassandra.index.sai.utils;
import java.io.Closeable;
-import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Iterator;
import javax.annotation.concurrent.NotThreadSafe;
@@ -29,7 +28,7 @@
/**
* Iterator to step through terms to obtain {@link PostingList} for the current term.
- *
+ * <p>
* Term enumerations are always ordered by their {@link ByteSource}.
*/
@NotThreadSafe
@@ -38,7 +37,7 @@
/**
* Get {@link PostingList} for the current term.
*/
- PostingList postings() throws IOException;
+ PostingList postings();
ByteBuffer getMinTerm();
diff --git a/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
index 90276d8..acdaf4c 100644
--- a/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
+++ b/src/java/org/apache/cassandra/index/sai/utils/TypeUtil.java
@@ -30,7 +30,6 @@
import com.googlecode.concurrenttrees.radix.ConcurrentRadixTree;
import org.apache.cassandra.cql3.statements.schema.IndexTarget;
import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.AsciiType;
import org.apache.cassandra.db.marshal.BooleanType;
import org.apache.cassandra.db.marshal.ByteBufferAccessor;
import org.apache.cassandra.db.marshal.CollectionType;
@@ -38,8 +37,9 @@
import org.apache.cassandra.db.marshal.DecimalType;
import org.apache.cassandra.db.marshal.InetAddressType;
import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
import org.apache.cassandra.db.marshal.ReversedType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.StringType;
import org.apache.cassandra.db.rows.Cell;
import org.apache.cassandra.db.rows.ComplexColumnData;
import org.apache.cassandra.index.sai.plan.Expression;
@@ -63,6 +63,12 @@
*/
public static final int DECIMAL_APPROXIMATION_BYTES = 24;
+ public static final int BIG_INTEGER_APPROXIMATION_BYTES = 20;
+
+ public static final int INET_ADDRESS_SIZE = 16;
+
+ public static final int DEFAULT_FIXED_LENGTH = 16;
+
private TypeUtil() {}
/**
@@ -130,6 +136,23 @@
return a == null ? b : (b == null || ByteComparable.compare(b, a, ByteComparable.Version.OSS50) < 0) ? a : b;
}
+ /**
+ * Returns the value length for the given {@link AbstractType}, selecting 16 for types
+ * that officially use VARIABLE_LENGTH but are, in fact, of a fixed length.
+ */
+ public static int fixedSizeOf(AbstractType<?> type)
+ {
+ if (type.isValueLengthFixed())
+ return type.valueLengthIfFixed();
+ else if (isInetAddress(type))
+ return INET_ADDRESS_SIZE;
+ else if (isBigInteger(type))
+ return BIG_INTEGER_APPROXIMATION_BYTES;
+ else if (isBigDecimal(type))
+ return DECIMAL_APPROXIMATION_BYTES;
+ return DEFAULT_FIXED_LENGTH;
+ }
+
public static AbstractType<?> cellValueType(ColumnMetadata columnMetadata, IndexTarget.Type indexType)
{
AbstractType<?> type = columnMetadata.type;
@@ -186,10 +209,39 @@
{
if (type instanceof InetAddressType || type instanceof IntegerType || type instanceof DecimalType)
return ByteSource.optionalFixedLength(ByteBufferAccessor.instance, value);
+ // The LongType.asComparableBytes uses variableLengthInteger which doesn't play well with
+ // the balanced tree because it is expecting fixed length data. So for SAI we use a optionalSignedFixedLengthNumber
+ // to keep all comparable values the same length
+ else if (type instanceof LongType)
+ return ByteSource.optionalSignedFixedLengthNumber(ByteBufferAccessor.instance, value);
return type.asComparableBytes(value, version);
}
/**
+ * Fills a byte array with the comparable bytes for a type.
+ * <p>
+ * This method expects a {@code value} parameter generated by calling {@link #asIndexBytes(ByteBuffer, AbstractType)}.
+ * It is not generally safe to pass the output of other serialization methods to this method. For instance, it is
+ * not generally safe to pass the output of {@link AbstractType#decompose(Object)} as the {@code value} parameter
+ * (there are certain types for which this is technically OK, but that doesn't hold for all types).
+ *
+ * @param value a value buffer returned by {@link #asIndexBytes(ByteBuffer, AbstractType)}
+ * @param type the type associated with the encoded {@code value} parameter
+ * @param bytes this method's output
+ */
+ public static void toComparableBytes(ByteBuffer value, AbstractType<?> type, byte[] bytes)
+ {
+ if (isInetAddress(type))
+ ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, INET_ADDRESS_SIZE);
+ else if (isBigInteger(type))
+ ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, BIG_INTEGER_APPROXIMATION_BYTES);
+ else if (isBigDecimal(type))
+ ByteBufferUtil.copyBytes(value, value.hasArray() ? value.arrayOffset() + value.position() : value.position(), bytes, 0, DECIMAL_APPROXIMATION_BYTES);
+ else
+ ByteSourceInverse.copyBytes(asComparableBytes(value, type, ByteComparable.Version.OSS50), bytes);
+ }
+
+ /**
* Translates the external value of specific types into a format used by the index.
*/
public static ByteBuffer asIndexBytes(ByteBuffer value, AbstractType<?> type)
@@ -307,7 +359,7 @@
private static boolean isIPv6(ByteBuffer address)
{
- return address.remaining() == 16;
+ return address.remaining() == INET_ADDRESS_SIZE;
}
/**
@@ -322,7 +374,7 @@
if (value.remaining() == 4)
{
int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position();
- ByteBuffer mapped = ByteBuffer.allocate(16);
+ ByteBuffer mapped = ByteBuffer.allocate(INET_ADDRESS_SIZE);
System.arraycopy(IPV4_PREFIX, 0, mapped.array(), 0, IPV4_PREFIX.length);
ByteBufferUtil.copyBytes(value, position, mapped, IPV4_PREFIX.length, value.remaining());
return mapped;
@@ -337,7 +389,7 @@
*
* The format of the encoding is:
*
- * The first 4 bytes contain the length of the {@link BigInteger} byte array
+ * The first 4 bytes contain the integer length of the {@link BigInteger} byte array
* with the top bit flipped for positive values.
*
* The remaining 16 bytes contain the 16 most significant bytes of the
@@ -350,18 +402,18 @@
{
int size = value.remaining();
int position = value.hasArray() ? value.arrayOffset() + value.position() : value.position();
- byte[] bytes = new byte[20];
- if (size < 16)
+ byte[] bytes = new byte[BIG_INTEGER_APPROXIMATION_BYTES];
+ if (size < BIG_INTEGER_APPROXIMATION_BYTES - Integer.BYTES)
{
ByteBufferUtil.copyBytes(value, position, bytes, bytes.length - size, size);
if ((bytes[bytes.length - size] & 0x80) != 0)
- Arrays.fill(bytes, 4, bytes.length - size, (byte)0xff);
+ Arrays.fill(bytes, Integer.BYTES, bytes.length - size, (byte)0xff);
else
- Arrays.fill(bytes, 4, bytes.length - size, (byte)0x00);
+ Arrays.fill(bytes, Integer.BYTES, bytes.length - size, (byte)0x00);
}
else
{
- ByteBufferUtil.copyBytes(value, position, bytes, 4, 16);
+ ByteBufferUtil.copyBytes(value, position, bytes, Integer.BYTES, BIG_INTEGER_APPROXIMATION_BYTES - Integer.BYTES);
}
if ((bytes[4] & 0x80) != 0)
{
@@ -380,16 +432,16 @@
*/
public static boolean isLiteral(AbstractType<?> type)
{
- return isUTF8OrAscii(type) || isCompositeOrFrozen(type) || baseType(type) instanceof BooleanType;
+ return isString(type) || isCompositeOrFrozen(type) || baseType(type) instanceof BooleanType;
}
/**
- * Returns <code>true</code> if given {@link AbstractType} is UTF8 or Ascii
+ * Returns <code>true</code> if given {@link AbstractType} is based on a string, e.g. UTF8 or Ascii
*/
- public static boolean isUTF8OrAscii(AbstractType<?> type)
+ public static boolean isString(AbstractType<?> type)
{
type = baseType(type);
- return type instanceof UTF8Type || type instanceof AsciiType;
+ return type instanceof StringType;
}
/**
diff --git a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
index 16b6679..4bf9d8c 100644
--- a/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
+++ b/src/java/org/apache/cassandra/utils/bytecomparable/ByteSourceInverse.java
@@ -402,6 +402,19 @@
return readBytes(byteSource, INITIAL_BUFFER_CAPACITY);
}
+ public static void copyBytes(ByteSource byteSource, byte[] bytes)
+ {
+ int readBytes = 0;
+
+ int data;
+ while ((data = byteSource.next()) != ByteSource.END_OF_STREAM)
+ {
+ if (bytes.length == readBytes)
+ throw new ArrayIndexOutOfBoundsException(String.format("Number of bytes read, %d, exceeds the buffer size of %d.", readBytes + 1, bytes.length));
+ bytes[readBytes++] = (byte) data;
+ }
+ }
+
/**
* Ensures the given buffer has capacity for taking data with the given length - if it doesn't, it returns a copy
* of the buffer, but with double the capacity.
diff --git a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
index 3069e80..6b8217f 100644
--- a/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
+++ b/test/distributed/org/apache/cassandra/distributed/test/sai/IndexStreamingTest.java
@@ -19,9 +19,13 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import java.util.Collections;
+import java.util.List;
import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.distributed.Cluster;
@@ -36,19 +40,16 @@
import static org.assertj.core.api.Assertions.assertThat;
+@RunWith(Parameterized.class)
public class IndexStreamingTest extends TestBaseImpl
{
// streaming sends events every 65k, so need to make sure that the files are larger than this to hit
// all cases of the vtable - hence we add a big enough blob column
private static final ByteBuffer BLOB = ByteBuffer.wrap(new byte[1 << 16]);
- private static final int NUM_COMPONENTS;
static
{
DatabaseDescriptor.clientInitialization();
- NUM_COMPONENTS = sstableStreamingComponentsCount()
- + V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size()
- + V1OnDiskFormat.LITERAL_COMPONENTS.size();
}
private static int sstableStreamingComponentsCount()
@@ -60,42 +61,62 @@
.count() - 1; // -1 because we don't include the compression component
}
- @Test
- public void zeroCopy() throws IOException
+ @SuppressWarnings("DefaultAnnotationParam")
+ @Parameterized.Parameter(0)
+ public boolean isLiteral;
+ @Parameterized.Parameter(1)
+ public boolean isZeroCopyStreaming;
+
+ @Parameterized.Parameters(name = "isLiteral={0}, isZeroCopyStreaming={1}")
+ public static List<Object[]> data()
{
- test(true);
+ List<Object[]> result = new ArrayList<>();
+ result.add(new Object[]{ true, true });
+ result.add(new Object[]{ true, false });
+ result.add(new Object[]{ false, true });
+ result.add(new Object[]{ false, false });
+ return result;
}
@Test
- public void notZeroCopy() throws IOException
- {
- test(false);
- }
-
- private void test(boolean zeroCopyStreaming) throws IOException
+ public void testIndexComponentStreaming() throws IOException
{
try (Cluster cluster = init(Cluster.build(2)
.withConfig(c -> c.with(Feature.values())
- .set("stream_entire_sstables", zeroCopyStreaming)
+ .set("stream_entire_sstables", isZeroCopyStreaming)
.set("streaming_slow_events_log_timeout", "0s"))
.start()))
{
cluster.schemaChange(withKeyspace(
- "CREATE TABLE %s.test (pk int PRIMARY KEY, v text, b blob) WITH compression = { 'enabled' : false };"
+ "CREATE TABLE %s.test (pk int PRIMARY KEY, literal text, numeric int, b blob) WITH compression = { 'enabled' : false };"
));
- cluster.schemaChange(withKeyspace(
- "CREATE CUSTOM INDEX ON %s.test(v) USING 'StorageAttachedIndex';"
- ));
+
+ int num_components = isLiteral ? sstableStreamingComponentsCount() +
+ V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size() +
+ V1OnDiskFormat.LITERAL_COMPONENTS.size()
+ : sstableStreamingComponentsCount() +
+ V1OnDiskFormat.PER_SSTABLE_COMPONENTS.size() +
+ V1OnDiskFormat.NUMERIC_COMPONENTS.size();
+
+ if (isLiteral)
+ cluster.schemaChange(withKeyspace(
+ "CREATE CUSTOM INDEX ON %s.test(literal) USING 'StorageAttachedIndex';"
+ ));
+ else
+ cluster.schemaChange(withKeyspace(
+ "CREATE CUSTOM INDEX ON %s.test(numeric) USING 'StorageAttachedIndex';"
+ ));
+
cluster.stream().forEach(i ->
i.nodetoolResult("disableautocompaction", KEYSPACE).asserts().success()
);
IInvokableInstance first = cluster.get(1);
IInvokableInstance second = cluster.get(2);
long sstableCount = 10;
- long expectedFiles = zeroCopyStreaming ? sstableCount * NUM_COMPONENTS : sstableCount;
+ long expectedFiles = isZeroCopyStreaming ? sstableCount * num_components : sstableCount;
for (int i = 0; i < sstableCount; i++)
{
- first.executeInternal(withKeyspace("insert into %s.test(pk, v, b) values (?, ?, ?)"), i, "v" + i, BLOB);
+ first.executeInternal(withKeyspace("insert into %s.test(pk, literal, numeric, b) values (?, ?, ?, ?)"), i, "v" + i, i, BLOB);
first.flush(KEYSPACE);
}
@@ -146,7 +167,8 @@
for (int i = 0; i < sstableCount; i++)
{
- Object[][] rs = second.executeInternal(withKeyspace("select pk from %s.test where v = ?"), "v" + i);
+ Object[][] rs = isLiteral ? second.executeInternal(withKeyspace("select pk from %s.test where literal = ?"), "v" + i)
+ : second.executeInternal(withKeyspace("select pk from %s.test where numeric = ?"), i);
assertThat(rs.length).isEqualTo(1);
assertThat(rs[0][0]).isEqualTo(i);
}
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index d2281cb..afdcfaf 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -778,6 +778,13 @@
disableCompaction(KEYSPACE);
}
+ public void disableCompaction(String keyspace, String table)
+ {
+ ColumnFamilyStore store = getColumnFamilyStore(keyspace, table);
+ if (store != null)
+ store.disableAutoCompaction();
+ }
+
public void enableCompaction(String keyspace)
{
ColumnFamilyStore store = getCurrentColumnFamilyStore(keyspace);
@@ -1572,7 +1579,7 @@
* Executes the provided query using the {@link ClientState#forInternalCalls()} as the expected ClientState. Note:
* this means permissions checking will not apply and queries will proceed regardless of role or guardrails.
*/
- protected UntypedResultSet executeFormattedQuery(String query, Object... values)
+ public UntypedResultSet executeFormattedQuery(String query, Object... values)
{
UntypedResultSet rs;
if (usePrepared)
diff --git a/test/unit/org/apache/cassandra/index/sai/SAITester.java b/test/unit/org/apache/cassandra/index/sai/SAITester.java
index db8a6ea..903ef1b 100644
--- a/test/unit/org/apache/cassandra/index/sai/SAITester.java
+++ b/test/unit/org/apache/cassandra/index/sai/SAITester.java
@@ -24,8 +24,6 @@
import java.io.IOException;
import java.io.OutputStream;
import java.io.RandomAccessFile;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.channels.FileChannel;
@@ -61,6 +59,7 @@
import com.datastax.driver.core.ResultSet;
import com.datastax.driver.core.Session;
import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.config.CassandraRelevantProperties;
import org.apache.cassandra.cql3.CQLTester;
import org.apache.cassandra.cql3.ColumnIdentifier;
import org.apache.cassandra.cql3.UntypedResultSet;
@@ -79,7 +78,6 @@
import org.apache.cassandra.index.sai.disk.format.Version;
import org.apache.cassandra.index.sai.disk.v1.V1OnDiskFormat;
import org.apache.cassandra.index.sai.disk.v1.segment.SegmentBuilder;
-import org.apache.cassandra.index.sai.utils.NamedMemoryLimiter;
import org.apache.cassandra.index.sai.utils.PrimaryKey;
import org.apache.cassandra.index.sai.utils.ResourceLeakDetector;
import org.apache.cassandra.inject.Injection;
@@ -114,11 +112,12 @@
protected static final String CREATE_KEYSPACE_TEMPLATE = "CREATE KEYSPACE IF NOT EXISTS %s WITH replication = " +
"{'class': 'SimpleStrategy', 'replication_factor': '1'}";
- protected static final String CREATE_TABLE_TEMPLATE = "CREATE TABLE %s (id1 TEXT PRIMARY KEY, v1 TEXT) WITH compaction = " +
+ protected static final String CREATE_TABLE_TEMPLATE = "CREATE TABLE %s (id1 TEXT PRIMARY KEY, v1 INT, v2 TEXT) WITH compaction = " +
"{'class' : 'SizeTieredCompactionStrategy', 'enabled' : false }";
protected static final String CREATE_INDEX_TEMPLATE = "CREATE CUSTOM INDEX IF NOT EXISTS ON %%s(%s) USING 'StorageAttachedIndex'";
protected static final ColumnIdentifier V1_COLUMN_IDENTIFIER = ColumnIdentifier.getInterned("v1", true);
+ protected static final ColumnIdentifier V2_COLUMN_IDENTIFIER = ColumnIdentifier.getInterned("v2", true);
protected static final Injections.Counter indexBuildCounter = Injections.newCounter("IndexBuildCounter")
.add(newInvokePoint().onClass(CompactionManager.class)
@@ -162,6 +161,9 @@
public void removeAllInjections()
{
Injections.deleteAll();
+ CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.reset();
+ CassandraRelevantProperties.SAI_POSTINGS_SKIP.reset();
+ V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMITER.setLimitBytes(V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMIT);
}
public static Randomization getRandom()
@@ -460,17 +462,23 @@
assertTrue(indexFiles().isEmpty());
}
- protected void verifyIndexFiles(IndexContext literalIndexContext, int literalFiles)
+ protected void verifyIndexFiles(IndexContext numericIndexContext, IndexContext literalIndexContext, int numericFiles, int literalFiles)
{
- verifyIndexFiles(literalIndexContext,
+ verifyIndexFiles(numericIndexContext,
+ literalIndexContext,
+ Math.max(numericFiles, literalFiles),
+ numericFiles,
literalFiles,
- literalFiles,
+ numericFiles,
literalFiles);
}
- protected void verifyIndexFiles(IndexContext literalIndexContext,
+ protected void verifyIndexFiles(IndexContext numericIndexContext,
+ IndexContext literalIndexContext,
int perSSTableFiles,
+ int numericFiles,
int literalFiles,
+ int numericCompletionMarkers,
int literalCompletionMarkers)
{
Set<File> indexFiles = indexFiles();
@@ -494,6 +502,20 @@
assertEquals(stringIndexFiles.toString(), literalFiles, stringIndexFiles.size());
}
}
+
+ if (numericIndexContext != null)
+ {
+ for (IndexComponent indexComponent : Version.LATEST.onDiskFormat().perColumnIndexComponents(numericIndexContext))
+ {
+ Set<File> numericIndexFiles = componentFiles(indexFiles,
+ SSTableFormat.Components.Types.CUSTOM.createComponent(Version.LATEST.fileNameFormatter().format(indexComponent,
+ numericIndexContext)));
+ if (isBuildCompletionMarker(indexComponent))
+ assertEquals(numericCompletionMarkers, numericIndexFiles.size());
+ else
+ assertEquals(numericIndexFiles.toString(), numericFiles, numericIndexFiles.size());
+ }
+ }
}
protected void verifySSTableIndexes(String indexName, int count)
@@ -730,19 +752,15 @@
}
}
- protected static void setSegmentWriteBufferSpace(final int segmentSize) throws Exception
+ protected static void setBDKPostingsWriterSizing(int minimumPostingsLeaves, int postingsSkip)
{
- NamedMemoryLimiter limiter = (NamedMemoryLimiter) V1OnDiskFormat.class.getDeclaredField("SEGMENT_BUILD_MEMORY_LIMITER").get(null);
- Field limitBytes = limiter.getClass().getDeclaredField("limitBytes");
- limitBytes.setAccessible(true);
- Field modifiersField = Field.class.getDeclaredField("modifiers");
- modifiersField.setAccessible(true);
- modifiersField.setInt(limitBytes, limitBytes.getModifiers() & ~Modifier.FINAL);
- limitBytes.set(limiter, segmentSize);
- limitBytes = V1OnDiskFormat.class.getDeclaredField("SEGMENT_BUILD_MEMORY_LIMIT");
- limitBytes.setAccessible(true);
- modifiersField.setInt(limitBytes, limitBytes.getModifiers() & ~Modifier.FINAL);
- limitBytes.set(limiter, segmentSize);
+ CassandraRelevantProperties.SAI_MINIMUM_POSTINGS_LEAVES.setString(Integer.toString(minimumPostingsLeaves));
+ CassandraRelevantProperties.SAI_POSTINGS_SKIP.setString(Integer.toString(postingsSkip));
+ }
+
+ protected static void setSegmentWriteBufferSpace(final int segmentSize)
+ {
+ V1OnDiskFormat.SEGMENT_BUILD_MEMORY_LIMITER.setLimitBytes(segmentSize);
}
protected String getSingleTraceStatement(Session session, String query, String contains)
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java b/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
index e9287e1..654c3e7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/BaseDataModel.java
@@ -167,6 +167,11 @@
tester.createTable(String.format(template, KEYSPACE, nonIndexedTable, keyColumnDefs, normalColumnDefs, primaryKey));
}
+ public void truncateTables(Executor tester) throws Throwable
+ {
+ executeLocal(tester, "TRUNCATE TABLE %s");
+ }
+
public void createIndexes(Executor tester) throws Throwable
{
createIndexes(tester, columns);
@@ -189,6 +194,24 @@
}
}
+ public void flush(Executor tester) throws Throwable
+ {
+ tester.flush(KEYSPACE, indexedTable);
+ tester.flush(KEYSPACE, nonIndexedTable);
+ }
+
+ public void disableCompaction(Executor tester) throws Throwable
+ {
+ tester.disableCompaction(KEYSPACE, indexedTable);
+ tester.disableCompaction(KEYSPACE, nonIndexedTable);
+ }
+
+ public void compact(Executor tester) throws Throwable
+ {
+ tester.compact(KEYSPACE, indexedTable);
+ tester.compact(KEYSPACE, nonIndexedTable);
+ }
+
public void insertRows(Executor tester) throws Throwable
{
String template = "INSERT INTO %%s (%s, %s) VALUES (%s, %s)";
@@ -210,6 +233,42 @@
}
}
+ public void updateCells(Executor tester) throws Throwable
+ {
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0", BIGINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 1", BOOLEAN_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 2", DATE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 3", DOUBLE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 4", FLOAT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 5", INET_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 6", INT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 7", SMALLINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 8", TINYINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 9", TEXT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 10", TIME_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 11", TIMESTAMP_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 12", UUID_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 13", TIMEUUID_COLUMN));
+ }
+
+ public void deleteCells(Executor tester) throws Throwable
+ {
+ for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+ {
+ executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s", NORMAL_COLUMNS.get(i).left, i));
+ }
+ }
+
+ public void deleteRows(Executor tester) throws Throwable
+ {
+ String template = "DELETE FROM %%s WHERE p = %d";
+
+ for (int deleted : deletable())
+ {
+ executeLocal(tester, String.format(template, deleted));
+ }
+ }
+
public void executeLocal(Executor tester, String query, Object... values) throws Throwable
{
tester.executeLocal(formatIndexedQuery(query), values);
@@ -270,6 +329,45 @@
}
@Override
+ public void updateCells(Executor tester) throws Throwable
+ {
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0 AND c = 0", BIGINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 0 AND c = 1", BOOLEAN_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 1 AND c = 0", DATE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 1 AND c = 1", DOUBLE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 2 AND c = 0", FLOAT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 2 AND c = 1", INET_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 3 AND c = 0", INT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 3 AND c = 1", SMALLINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 4 AND c = 0", TINYINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 4 AND c = 1", TEXT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 5 AND c = 0", TIME_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 5 AND c = 1", TIMESTAMP_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 6 AND c = 0", UUID_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 6 AND c = 1", TIMEUUID_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 1896 WHERE p = 7", STATIC_INT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 2020 WHERE p = 100", STATIC_INT_COLUMN)); // static only
+ }
+
+ @Override
+ public void deleteCells(Executor tester) throws Throwable
+ {
+ for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+ {
+ String[] primaryKey = keys.get(i).split(",");
+ executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s AND c = %s", NORMAL_COLUMNS.get(i).left, primaryKey[0], primaryKey[1]));
+ }
+ }
+
+ @Override
+ public void deleteRows(Executor tester) throws Throwable
+ {
+ executeLocal(tester, "DELETE FROM %s WHERE p = 2 AND c = 0");
+ executeLocal(tester, "DELETE FROM %s WHERE p = 4 AND c = 0");
+ executeLocal(tester, "DELETE FROM %s WHERE p = 6");
+ }
+
+ @Override
protected Set<Integer> deletable()
{
return Sets.newHashSet(4, 8, 12, 13, 100);
@@ -312,6 +410,45 @@
}
@Override
+ public void updateCells(Executor tester) throws Throwable
+ {
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p1 = 0 AND p2 = 0", BIGINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p1 = 0 AND p2 = 1", BOOLEAN_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p1 = 1 AND p2 = 0", DATE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p1 = 1 AND p2 = 1", DOUBLE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p1 = 2 AND p2 = 0", FLOAT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p1 = 2 AND p2 = 1", INET_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p1 = 3 AND p2 = 0", INT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p1 = 3 AND p2 = 1", SMALLINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p1 = 4 AND p2 = 0", TINYINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p1 = 4 AND p2 = 2", TEXT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p1 = 5 AND p2 = 3", TIME_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p1 = 5 AND p2 = 1", TIMESTAMP_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p1 = 6 AND p2 = 0", UUID_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p1 = 6 AND p2 = 1", TIMEUUID_COLUMN));
+ }
+
+ @Override
+ public void deleteCells(Executor tester) throws Throwable
+ {
+ for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+ {
+ String[] primaryKey = keys.get(i).split(",");
+ executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p1 = %s AND p2 = %s",
+ NORMAL_COLUMNS.get(i).left, primaryKey[0], primaryKey[1]));
+ }
+ }
+
+ @Override
+ public void deleteRows(Executor tester) throws Throwable
+ {
+ executeLocal(tester, "DELETE FROM %s WHERE p1 = 2 AND p2 = 0");
+ executeLocal(tester, "DELETE FROM %s WHERE p1 = 4 AND p2 = 1");
+ executeLocal(tester, "DELETE FROM %s WHERE p1 = 6 AND p2 = 2");
+ executeLocal(tester, "DELETE FROM %s WHERE p1 = 8 AND p2 = 0");
+ }
+
+ @Override
protected Set<Integer> deletable()
{
// already overwrites {@code deleteRows()}
@@ -330,6 +467,33 @@
this.keys = new CompoundPrimaryKeyList(rows.size(), 1);
}
+ @Override
+ public void updateCells(Executor tester) throws Throwable
+ {
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 9700000000 WHERE p = 0 AND c = 0", BIGINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = false WHERE p = 1 AND c = 0", BOOLEAN_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2018-03-10' WHERE p = 2 AND c = 0", DATE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 8788.06 WHERE p = 3 AND c = 0", DOUBLE_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 2.9 WHERE p = 4 AND c = 0", FLOAT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '205.204.196.65' WHERE p = 5 AND c = 0", INET_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 27429638 WHERE p = 6 AND c = 0", INT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 31 WHERE p = 7 AND c = 0", SMALLINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 116 WHERE p = 8 AND c = 0", TINYINT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 'State of Michigan' WHERE p = 9 AND c = 0", TEXT_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '00:20:26' WHERE p = 10 AND c = 0", TIME_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = '2009-07-16T00:00:00' WHERE p = 11 AND c = 0", TIMESTAMP_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = e37394dc-d17b-11e8-a8d5-f2801f1b9fd1 WHERE p = 12 AND c = 0", UUID_COLUMN));
+ executeLocal(tester, String.format("UPDATE %%s SET %s = 1fc81a4c-d17d-11e8-a8d5-f2801f1b9fd1 WHERE p = 13 AND c = 0", TIMEUUID_COLUMN));
+ }
+
+ @Override
+ public void deleteCells(Executor tester) throws Throwable
+ {
+ for (int i = 0; i < NORMAL_COLUMNS.size(); i++)
+ {
+ executeLocal(tester, String.format("DELETE %s FROM %%s WHERE p = %s AND c = 0", NORMAL_COLUMNS.get(i).left, i));
+ }
+ }
}
static class SimplePrimaryKeyList extends ForwardingList<String>
@@ -382,6 +546,12 @@
{
void createTable(String statement);
+ void flush(String keyspace, String table);
+
+ void compact(String keyspace, String table);
+
+ void disableCompaction(String keyspace, String table);
+
void waitForIndexQueryable(String keyspace, String index);
void executeLocal(String query, Object...values) throws Throwable;
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java b/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
index 7573fc9..cd8a047 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/IndexQuerySupport.java
@@ -92,11 +92,100 @@
{
dataModel.createTables(executor);
+ dataModel.disableCompaction(executor);
+
dataModel.createIndexes(executor);
// queries against Memtable adjacent in-memory indexes
dataModel.insertRows(executor);
executeQueries(dataModel, executor, sets);
+
+ // queries with Memtable flushed to SSTable on disk
+ dataModel.flush(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries across memory and disk indexes
+ dataModel.insertRows(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries w/ multiple SSTable indexes
+ dataModel.flush(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries after compacting to a single SSTable index
+ dataModel.compact(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries against Memtable updates and the existing SSTable index
+ dataModel.updateCells(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries against the newly flushed SSTable index and the existing SSTable index
+ dataModel.flush(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries after compacting updates into to a single SSTable index
+ dataModel.compact(executor);
+ executeQueries(dataModel, executor, sets);
+ }
+
+ public static void rowDeletions(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
+ {
+ dataModel.createTables(executor);
+
+ dataModel.disableCompaction(executor);
+
+ dataModel.createIndexes(executor);
+ dataModel.insertRows(executor);
+ dataModel.flush(executor);
+ dataModel.compact(executor);
+
+ // baseline queries
+ executeQueries(dataModel, executor, sets);
+
+ // queries against Memtable deletes and the existing SSTable index
+ dataModel.deleteRows(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries against the newly flushed SSTable index and the existing SSTable index
+ dataModel.flush(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries after compacting deletes into to a single SSTable index
+ dataModel.compact(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // truncate, reload, and verify that the load is clean
+ dataModel.truncateTables(executor);
+ dataModel.insertRows(executor);
+ executeQueries(dataModel, executor, sets);
+ }
+
+ public static void cellDeletions(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
+ {
+ dataModel.createTables(executor);
+
+ dataModel.disableCompaction(executor);
+
+ dataModel.createIndexes(executor);
+ dataModel.insertRows(executor);
+ dataModel.flush(executor);
+ dataModel.compact(executor);
+
+ // baseline queries
+ executeQueries(dataModel, executor, sets);
+
+ // queries against Memtable deletes and the existing SSTable index
+ dataModel.deleteCells(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries against the newly flushed SSTable index and the existing SSTable index
+ dataModel.flush(executor);
+ executeQueries(dataModel, executor, sets);
+
+ // queries after compacting deletes into to a single SSTable index
+ dataModel.compact(executor);
+ executeQueries(dataModel, executor, sets);
}
public static void timeToLive(BaseDataModel.Executor executor, BaseDataModel dataModel, List<BaseQuerySet> sets) throws Throwable
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java
new file mode 100644
index 0000000..6b42358
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/QueryCellDeletionsTest.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Test;
+
+public class QueryCellDeletionsTest extends AbstractQueryTester
+{
+ @Test
+ public void testCellDeletions() throws Throwable
+ {
+ IndexQuerySupport.cellDeletions(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java
new file mode 100644
index 0000000..a5a843d
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/QueryRowDeletionsTest.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Test;
+
+public class QueryRowDeletionsTest extends AbstractQueryTester
+{
+ @Test
+ public void testRowDeletions() throws Throwable
+ {
+ IndexQuerySupport.rowDeletions(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java b/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
index 1e86eb4..ea20311 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/SingleNodeExecutor.java
@@ -43,6 +43,24 @@
}
@Override
+ public void flush(String keyspace, String table)
+ {
+ tester.flush(keyspace, table);
+ }
+
+ @Override
+ public void compact(String keyspace, String table)
+ {
+ tester.compact(keyspace, table);
+ }
+
+ @Override
+ public void disableCompaction(String keyspace, String table)
+ {
+ tester.disableCompaction(keyspace, table);
+ }
+
+ @Override
public void waitForIndexQueryable(String keyspace, String index)
{
tester.waitForIndexQueryable(keyspace, index);
@@ -51,7 +69,7 @@
@Override
public void executeLocal(String query, Object... values) throws Throwable
{
- tester.execute(query, values);
+ tester.executeFormattedQuery(query, values);
}
@Override
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java b/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
index 52d364c..fab1161 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/StorageAttachedIndexDDLTest.java
@@ -47,6 +47,7 @@
import org.apache.cassandra.db.compaction.CompactionManager;
import org.apache.cassandra.db.compaction.OperationType;
import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.db.marshal.ReversedType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.index.Index;
@@ -303,7 +304,8 @@
{
createTable(CREATE_TABLE_TEMPLATE);
- createIndex("CREATE CUSTOM INDEX ON %s(v1) USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': 'CONTAINS',\n" +
+ createIndex("CREATE CUSTOM INDEX ON %s(v1) USING 'org.apache.cassandra.index.sasi.SASIIndex'");
+ createIndex("CREATE CUSTOM INDEX ON %s(v2) USING 'org.apache.cassandra.index.sasi.SASIIndex' WITH OPTIONS = {'mode': 'CONTAINS',\n" +
"'analyzer_class': 'org.apache.cassandra.index.sasi.analyzer.StandardAnalyzer',\n" +
"'tokenization_enable_stemming': 'true',\n" +
"'tokenization_locale': 'en',\n" +
@@ -311,9 +313,12 @@
"'analyzed': 'true',\n" +
"'tokenization_normalize_lowercase': 'true'};");
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0' ALLOW FILTERING");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(1, rows.all().size());
+
+ rows = executeNet("SELECT id1 FROM %s WHERE v2 like '0'");
assertEquals(1, rows.all().size());
}
@@ -363,11 +368,12 @@
int rowCount = 10;
for (int i = 0; i < rowCount; i++)
- execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ waitForTableIndexesQueryable();
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
assertEquals(rowCount, rows.all().size());
}
@@ -407,12 +413,12 @@
int rowCount = 10;
for (int i = 0; i < rowCount; i++)
- execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
Injections.inject(forceFlushPause);
createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
- assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'")).isInstanceOf(ReadFailureException.class);
+ assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0")).isInstanceOf(ReadFailureException.class);
}
@Test
@@ -423,7 +429,7 @@
int rowCount = 10;
for (int i = 0; i < rowCount; i++)
- execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', " + i + ", '0')");
flush();
Injections.inject(failSAIInitialializaion);
@@ -431,7 +437,7 @@
waitForAssert(() -> assertEquals(1, indexBuildCounter.get()));
waitForCompactions();
- assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'")).isInstanceOf(ReadFailureException.class);
+ assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0")).isInstanceOf(ReadFailureException.class);
}
@Test
@@ -439,28 +445,50 @@
{
createTable(CREATE_TABLE_TEMPLATE);
- String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String numericIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+ IndexContext numericIndexContext = createIndexContext(numericIndexName, Int32Type.instance);
IndexContext literalIndexContext = createIndexContext(literalIndexName, UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 0);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 0);
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
flush();
- verifyIndexFiles(literalIndexContext, 1);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(1, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(1, rows.all().size());
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0')");
flush();
- verifyIndexFiles(literalIndexContext, 2);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+ verifySSTableIndexes(numericIndexName, 2, 2);
verifySSTableIndexes(literalIndexName, 2, 2);
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(2, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(2, rows.all().size());
+ dropIndex("DROP INDEX %s." + numericIndexName);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 2);
+ verifySSTableIndexes(numericIndexName, 2, 0);
+ verifySSTableIndexes(literalIndexName, 2, 2);
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+ assertEquals(2, rows.all().size());
+
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('2', 2, '0')");
+ flush();
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 3);
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+ assertEquals(3, rows.all().size());
+
dropIndex("DROP INDEX %s." + literalIndexName);
- verifyIndexFiles(literalIndexContext, 0);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 0, 0);
+ verifySSTableIndexes(numericIndexName, 0);
verifySSTableIndexes(literalIndexName, 0);
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero on drop.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -469,20 +497,25 @@
createTable(CREATE_TABLE_TEMPLATE);
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
verifyNoIndexFiles();
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 2);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(2, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(2, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -491,20 +524,30 @@
createTable(CREATE_TABLE_TEMPLATE);
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
verifyNoIndexFiles();
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 2);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext, null, 2, 0);
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
assertEquals(2, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(2, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+ assertEquals(2, rows.all().size());
+
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -513,28 +556,36 @@
createTable(CREATE_TABLE_TEMPLATE);
disableCompaction(KEYSPACE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
- verifyIndexFiles(literalIndexContext, 1);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(1, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(1, rows.all().size());
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
- verifyIndexFiles(literalIndexContext, 2);
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 2);
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(2, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(2, rows.all().size());
compact();
- waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1));
+ waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1));
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(2, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(2, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero after compaction.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -556,6 +607,7 @@
if (!concurrentTruncate)
{
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
}
// create 100 rows, half in sstable and half in memtable
@@ -564,12 +616,13 @@
{
if (i == num / 2)
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', 0, '0');");
}
if (concurrentTruncate)
{
- createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
truncate(true);
waitForTableIndexesQueryable();
}
@@ -582,8 +635,10 @@
// verify index-view-manager has been cleaned up
verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 0);
+ verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 0);
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero after truncate.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -591,41 +646,51 @@
{
// prepare schema and data
createTable(CREATE_TABLE_TEMPLATE);
- String literalIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String numericIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String stringIndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
for (CorruptionType corruptionType : CorruptionType.values())
{
- verifyRebuildCorruptedFiles(literalIndexName, corruptionType, false);
- verifyRebuildCorruptedFiles(literalIndexName, corruptionType, true);
+ verifyRebuildCorruptedFiles(numericIndexName, stringIndexName, corruptionType, false);
+ verifyRebuildCorruptedFiles(numericIndexName, stringIndexName, corruptionType, true);
}
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero following rebuild.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
- private void verifyRebuildCorruptedFiles(String literalIndexName,
+ private void verifyRebuildCorruptedFiles(String numericIndexName,
+ String stringIndexName,
CorruptionType corruptionType,
boolean rebuild) throws Throwable
{
- IndexContext LiteralIndexContext = createIndexContext(literalIndexName, UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(numericIndexName, Int32Type.instance);
+ IndexContext stringIndexContext = createIndexContext(stringIndexName, UTF8Type.instance);
for (IndexComponent component : Version.LATEST.onDiskFormat().perSSTableIndexComponents())
- verifyRebuildIndexComponent(LiteralIndexContext, component, null, corruptionType, rebuild);
+ verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, null, corruptionType, true, true, rebuild);
- for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(LiteralIndexContext))
- verifyRebuildIndexComponent(LiteralIndexContext, component, LiteralIndexContext, corruptionType, rebuild);
+ for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(numericIndexContext))
+ verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, numericIndexContext, corruptionType, false, true, rebuild);
+
+ for (IndexComponent component : Version.LATEST.onDiskFormat().perColumnIndexComponents(stringIndexContext))
+ verifyRebuildIndexComponent(numericIndexContext, stringIndexContext, component, stringIndexContext, corruptionType, true, false, rebuild);
}
- private void verifyRebuildIndexComponent(IndexContext literalIndexContext,
+ private void verifyRebuildIndexComponent(IndexContext numericIndexContext,
+ IndexContext stringIndexContext,
IndexComponent component,
IndexContext corruptionContext,
CorruptionType corruptionType,
+ boolean failedStringIndex,
+ 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.
@@ -641,11 +706,15 @@
int rowCount = 2;
// initial verification
- verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
- verifyIndexFiles(literalIndexContext, 1, 1, 1);
- assertTrue(verifyChecksum(literalIndexContext));
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+ verifySSTableIndexes(stringIndexContext.getIndexName(), 1);
+ verifyIndexFiles(numericIndexContext, stringIndexContext, 1, 1, 1, 1, 1);
+ assertTrue(verifyChecksum(numericIndexContext));
+ assertTrue(verifyChecksum(numericIndexContext));
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(rowCount, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(rowCount, rows.all().size());
// corrupt file
@@ -656,19 +725,17 @@
// If we are removing completion markers then the rest of the components should still have
// valid checksums.
- boolean expectedLiteralState = isBuildCompletionMarker(component);
+ boolean expectedNumericState = !failedNumericIndex || isBuildCompletionMarker(component);
+ boolean expectedLiteralState = !failedStringIndex || isBuildCompletionMarker(component);
- assertEquals("Validation for " + component + " should be " + expectedLiteralState + " but was " + !expectedLiteralState,
- expectedLiteralState,
- validateComponents(literalIndexContext));
-
- assertEquals("Checksum verification for " + component + " should be " + expectedLiteralState + " but was " + !expectedLiteralState,
- expectedLiteralState,
- verifyChecksum(literalIndexContext));
+ assertEquals("Checksum verification for " + component + " should be " + expectedNumericState + " but was " + !expectedNumericState,
+ expectedNumericState,
+ verifyChecksum(numericIndexContext));
+ assertEquals(expectedLiteralState, verifyChecksum(stringIndexContext));
if (rebuild)
{
- rebuildIndexes(literalIndexContext.getIndexName());
+ rebuildIndexes(numericIndexContext.getIndexName(), stringIndexContext.getIndexName());
}
else
{
@@ -676,13 +743,25 @@
reloadSSTableIndex();
// Verify the index cannot be read:
- verifySSTableIndexes(literalIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, failedNumericIndex ? 0 : 1);
+ verifySSTableIndexes(stringIndexContext.getIndexName(), Version.LATEST.onDiskFormat().perSSTableIndexComponents().contains(component) ? 0 : 1, failedStringIndex ? 0 : 1);
try
{
// If the corruption is that a file is missing entirely, the index won't be marked non-queryable...
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
- assertEquals(0, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(failedNumericIndex ? 0 : rowCount, rows.all().size());
+ }
+ catch (ReadFailureException e)
+ {
+ // ...but most kind of corruption will result in the index being non-queryable.
+ }
+
+ try
+ {
+ // If the corruption is that a file is missing entirely, the index won't be marked non-queryable...
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
+ assertEquals(failedStringIndex ? 0 : rowCount, rows.all().size());
}
catch (ReadFailureException e)
{
@@ -694,10 +773,13 @@
}
// verify indexes are recovered
- verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
- verifyIndexFiles(literalIndexContext, 1, 1, 1);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
+ verifyIndexFiles(numericIndexContext, stringIndexContext, 1, 1, 1, 1, 1);
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(rowCount, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(rowCount, rows.all().size());
}
@@ -707,9 +789,9 @@
createTable(CREATE_TABLE_TEMPLATE);
disableCompaction(KEYSPACE);
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
// Inject failure
@@ -719,17 +801,16 @@
try
{
// Create a new index, which will actuate a build compaction and fail, but leave the node running...
- IndexContext literalIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
// two index builders running in different compaction threads because of parallelised index initial build
waitForAssert(() -> assertEquals(2, indexBuildCounter.get()));
waitForCompactionsFinished();
- // Only token/primary key files for the first SSTable in the compaction task should exist, while column-specific files are blown away:
- verifyIndexFiles(literalIndexContext, 2, 0, 0);
+ // Only token/offset files for the first SSTable in the compaction task should exist, while column-specific files are blown away:
+ verifyIndexFiles(numericIndexContext, null, 2, 0, 0, 0, 0);
- Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
-
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
finally
{
@@ -738,14 +819,14 @@
}
@Test
- public void verifyCleanupFailedTokenPrimaryKeyFiles() throws Throwable
+ public void verifyCleanupFailedPrimaryKeyFiles() throws Throwable
{
createTable(CREATE_TABLE_TEMPLATE);
disableCompaction(KEYSPACE);
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '0');");
flush();
// Inject failure
@@ -763,9 +844,8 @@
// SSTable-level token/offset file(s) should be removed, while column-specific files never existed:
verifyNoIndexFiles();
- Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
-
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
finally
{
@@ -779,7 +859,8 @@
createTable(CREATE_TABLE_TEMPLATE);
disableCompaction(KEYSPACE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
// flush empty index
execute("INSERT INTO %s (id1) VALUES ('0');");
@@ -788,19 +869,24 @@
execute("INSERT INTO %s (id1) VALUES ('1');");
flush();
- verifyIndexFiles(literalIndexContext, 2, 0, 2);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 0, 0, 2, 2);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(0, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(0, rows.all().size());
// compact empty index
compact();
- waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+ waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 0, 0, 1, 1));
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(0, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(0, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -854,24 +940,33 @@
disableCompaction(KEYSPACE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+ waitForTableIndexesQueryable();
populateData.run();
verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 2, 0);
- verifyIndexFiles(literalIndexContext, 2, 0, 2);
+ verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 2, 0);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 2, 0, 0, 2, 2);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(0, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(0, rows.all().size());
// compact empty index
compact();
verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), 1, 0);
- waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+ verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), 1, 0);
+ waitForAssert(() -> verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 0, 0, 1, 1));
- rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(0, rows.all().size());
+ rows = executeNet("SELECT id1 FROM %s WHERE v2='0'");
assertEquals(0, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
}
@Test
@@ -883,7 +978,7 @@
int num = 100;
for (int i = 0; i < num; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
}
flush();
@@ -892,7 +987,7 @@
.build();
Injections.inject(delayIndexBuilderCompletion);
- String indexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
waitForAssert(() -> assertEquals(1, delayIndexBuilderCompletion.getCount()));
dropIndex("DROP INDEX %s." + indexName);
@@ -908,9 +1003,10 @@
// create index again, it should succeed
indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ waitForTableIndexesQueryable();
verifySSTableIndexes(indexName, 1);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
assertEquals(num, rows.all().size());
}
@@ -926,7 +1022,7 @@
int sstable = 1;
for (int i = 0; i < num; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES ('" + i + "', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('" + i + "', 0, '0');");
}
flush();
@@ -936,7 +1032,7 @@
Injections.inject(delayIndexBuilderCompletion);
- IndexContext literalIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
waitForAssert(() -> assertTrue(getCompactionTasks() > 0), 1000, TimeUnit.MILLISECONDS);
@@ -944,6 +1040,8 @@
int attempt = 20;
while (getCompactionTasks() > 0 && attempt > 0)
{
+ System.out.println("Attempt " + attempt + " at stopping the compaction tasks");
+
// only interrupts active compactions, not pending compactions.
CompactionManager.instance.stopCompaction(OperationType.INDEX_BUILD.name());
// let blocked builder to continue, but still block pending builder threads
@@ -958,7 +1056,7 @@
delayIndexBuilderCompletion.disable();
// initial index builder should have stopped abruptly resulting in the index not being queryable
- verifyInitialIndexFailed(literalIndexContext.getIndexName());
+ verifyInitialIndexFailed(numericIndexContext.getIndexName());
Assertions.assertThat(getNotQueryableIndexes()).isNotEmpty();
ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
@@ -971,18 +1069,20 @@
assertTrue("Expect index build stopped", view.getIndexes().isEmpty());
}
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero on interrupted compactions.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
// rebuild index
- ColumnFamilyStore.rebuildSecondaryIndex(KEYSPACE, currentTable(), literalIndexContext.getIndexName());
+ ColumnFamilyStore.rebuildSecondaryIndex(KEYSPACE, currentTable(), numericIndexContext.getIndexName());
- verifyIndexFiles(literalIndexContext, sstable);
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1='0'");
+ verifyIndexFiles(numericIndexContext, null, sstable, 0);
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
assertEquals(num, rows.all().size());
- assertZeroSegmentBuilderUsage();
+ assertEquals("Segment memory limiter should revert to zero following rebuild.", 0L, getSegmentBufferUsedBytes());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
- assertTrue(verifyChecksum(literalIndexContext));
+ assertTrue(verifyChecksum(numericIndexContext));
}
@Test
@@ -992,7 +1092,7 @@
String index = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
- assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %%s WHERE expr(%s, '0')", index)))
+ assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %%s WHERE expr(%s, 0)", index)))
.isInstanceOf(InvalidQueryException.class)
.hasMessage(String.format(IndexRestrictions.CUSTOM_EXPRESSION_NOT_SUPPORTED, index));
}
@@ -1068,7 +1168,7 @@
assertEquals(Arrays.asList(2L, 1L), toSize.apply(iterator.next()));
}
- private void assertZeroSegmentBuilderUsage() throws Exception
+ private void assertZeroSegmentBuilderUsage()
{
assertEquals("Segment memory limiter should revert to zero.", 0L, getSegmentBufferUsedBytes());
assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java
new file mode 100644
index 0000000..39856bd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryCellDeletionsTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryCellDeletionsTest extends AbstractQueryTester
+{
+ @Before
+ public void setSegmentWriteBufferSpace() throws Throwable
+ {
+ setSegmentWriteBufferSpace(0);
+ }
+
+ @Test
+ public void testCellDeletions() throws Throwable
+ {
+ IndexQuerySupport.cellDeletions(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java
new file mode 100644
index 0000000..5a67230
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryRowDeletionsTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryRowDeletionsTest extends AbstractQueryTester
+{
+ @Before
+ public void setSegmentWriteBufferSpace() throws Throwable
+ {
+ setSegmentWriteBufferSpace(0);
+ }
+
+ @Test
+ public void testRowDeletions() throws Throwable
+ {
+ IndexQuerySupport.rowDeletions(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java
new file mode 100644
index 0000000..b89a8a8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryTimeToLiveTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryTimeToLiveTest extends AbstractQueryTester
+{
+ @Before
+ public void setSegmentWriteBufferSpace() throws Throwable
+ {
+ setSegmentWriteBufferSpace(0);
+ }
+
+ @Test
+ public void testTimeToLive() throws Throwable
+ {
+ IndexQuerySupport.timeToLive(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java
new file mode 100644
index 0000000..b32e1db
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/cql/TinySegmentQueryWriteLifecycleTest.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.cql;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Force generates segments due to a small RAM size on compaction, to test segment splitting
+ */
+public class TinySegmentQueryWriteLifecycleTest extends AbstractQueryTester
+{
+ @Before
+ public void setSegmentWriteBufferSpace() throws Throwable
+ {
+ setSegmentWriteBufferSpace(0);
+ }
+
+ @Test
+ public void testWriteLifecycle() throws Throwable
+ {
+ IndexQuerySupport.writeLifecycle(executor, dataModel, sets);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
index fdc60ed..485fd1f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/AsciiTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new DataSet.AsciiDataSet());
+ return generateParameters(new DataSet.AsciiDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
index 2062c68..603a87e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/BigintTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.BigintDataSet());
+ return generateParameters(new DataSet.BigintDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
index 4f2289e..472fdfd 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/BooleanTest.java
@@ -27,6 +27,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new DataSet.BooleanDataSet());
+ return generateParameters(new DataSet.BooleanDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
index 14bfedd..7e97717 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DateTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.DateDataSet());
+ return generateParameters(new DataSet.DateDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
index a4b7589..8dc1071 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DecimalTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.DecimalDataSet());
+ return generateParameters(new DataSet.DecimalDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
index 81ce202..4222d7f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/DoubleTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.DoubleDataSet());
+ return generateParameters(new DataSet.DoubleDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
index aa185a5..598f95c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/FloatTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.FloatDataSet());
+ return generateParameters(new DataSet.FloatDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java b/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
index 15acbda..b1d5408 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/IndexingTypeSupport.java
@@ -52,7 +52,7 @@
POST_BUILD_QUERY
}
- protected static Collection<Object[]> generateLiteralParameters(DataSet<?> dataset)
+ protected static Collection<Object[]> generateParameters(DataSet<?> dataset)
{
return Arrays.asList(new Object[][]
{
@@ -69,15 +69,6 @@
});
}
- protected static Collection<Object[]> generateNumericParameters(DataSet<?> dataset)
- {
- return Arrays.asList(new Object[][]
- {
- { dataset, true, Scenario.MEMTABLE_QUERY },
- { dataset, false, Scenario.MEMTABLE_QUERY },
- });
- }
-
@Before
public void createTable()
{
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
index 19c9fe4..a3c2796 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/InetTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.InetDataSet());
+ return generateParameters(new DataSet.InetDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
index 3fbcefd..a96f2b1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/IntTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.IntDataSet());
+ return generateParameters(new DataSet.IntDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
index c960ec4..617752e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/SmallintTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.SmallintDataSet());
+ return generateParameters(new DataSet.SmallintDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
index 474b7c2..7c40d8e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TextTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new DataSet.TextDataSet());
+ return generateParameters(new DataSet.TextDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
index a716d8f..316b371 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.TimeDataSet());
+ return generateParameters(new DataSet.TimeDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
index 37293c4..1992c7a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimestampTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.TimestampDataSet());
+ return generateParameters(new DataSet.TimestampDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
index d79e408..b5b7dee 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TimeuuidTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.TimeuuidDataSet());
+ return generateParameters(new DataSet.TimeuuidDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
index edabf52..118d5c7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/TinyintTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.TinyintDataSet());
+ return generateParameters(new DataSet.TinyintDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
index 4225ed0..ed11f0e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/UuidTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.UuidDataSet());
+ return generateParameters(new DataSet.UuidDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
index 6fbe977..5263380 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/VarintTest.java
@@ -26,6 +26,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new DataSet.VarintDataSet());
+ return generateParameters(new DataSet.VarintDataSet());
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
index 58ee912..03d4d22 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
index eda791e..c239267 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListDecimalTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.DecimalDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
index 4161cc9..8d3ab67 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
index 1864b96..98ac32e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/FrozenListVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenListDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
index d520751..22c4c9f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
index bafe565..44c856f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListBigintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.BigintDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.BigintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
index 97452db..a79841c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDateTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DateDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DateDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
index a718539..c52f17d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDecimalTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DecimalDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
index 0f9c0a4..64e1363 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListDoubleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DoubleDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.DoubleDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
index 533a238..8925e24 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFloatTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.FloatDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.FloatDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
index 7ef4155..c670acf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<List<Integer>> frozen = new CollectionDataSet.FrozenListDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
index 6033ea4..0138975 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListInetTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.InetDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.InetDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
index de3fed3..0f581aa 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
index 55182ce..3f6cb8b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListSmallintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.SmallintDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.SmallintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
index 7f38d27..098a57f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTextTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TextDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TextDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
index bdb0339..d443bf1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
index b1aa9a2..5797f53 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimestampTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimestampDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimestampDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
index dd7b8f1..087fc7e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTimeuuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeuuidDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TimeuuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
index 1ecda14..f4ed256 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListTinyintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TinyintDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.TinyintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
index c315961..49a4841 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListUuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.UuidDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.UuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
index 4b201ea..17df156 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/lists/ListVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.ListDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.ListDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
index f8df465..dc3faf7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
index 3d110bf..d17c2c3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapDecimalTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.DecimalDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
index 14ab4e2..4d1a393 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
index 6424fe0..a393d70 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/FrozenMapVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
index 6d56ee5..81650cf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
index 3af7fde..2e8e434 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapBigintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
index 13b5294..d5c2902 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDateTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DateDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DateDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
index 89f67e4..2511921 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDecimalTest.java
@@ -30,7 +30,7 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DecimalDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
index c758b00..67056b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapDoubleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DoubleDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.DoubleDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
index 3ed8435..4526b73 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
index 2930829..2f06531 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.MapEntriesDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
index 97e5d69..71bd9b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
index 121128f..852bc7d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapEntriesVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.MapEntriesDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
index 4a8b044..31e39c0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFloatTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.FloatDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.FloatDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
index 8787b96..b39a8d3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
index d455f84..62def5b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapInetTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.InetDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.InetDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
index cd0c7c9..5a33360 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
index aaaf818..5b0837b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
index 1ed21d5..e2c4c84 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.MapKeysDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.MapKeysDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
index dcd101b..6e75d15 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
index d6aae85..52c395a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapKeysVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.MapKeysDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
index 8ea194e..cd05883 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapSmallintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.SmallintDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.SmallintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
index e0058a7..29cf454 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTextTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TextDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TextDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
index 82504d9..9923457 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
index 40fe6ee..1f5cd8e 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimestampTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimestampDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimestampDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
index 26e0a7e..2a46698 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTimeuuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeuuidDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TimeuuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
index d8102c2..c832905 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapTinyintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TinyintDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.TinyintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
index 9f77a37..0963ee0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapUuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.UuidDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.UuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
index 4f3068a..88c8e30 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
index 9e82234..15e9dfb 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<Map<Integer, Integer>> frozen = new CollectionDataSet.FrozenMapValuesDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.MapValuesDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.MapValuesDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
index 5db3f0d..3bf3840 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
index c49baf5..e5077fb 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapValuesVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MapValuesDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
index a4fc1e0..d85880a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MapVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MapDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.MapDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
index e1eccd4..8c5f9f3 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
index 7d9af16..270f741 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
index 97d0144..9d02cf2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/maps/MultiMapVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.MultiMapDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
index 02b8f26..1672d22 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
index aec4946..591bce5 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetDecimalTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.DecimalDataSet()));
}
}
\ No newline at end of file
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
index 4db6b9f..6f43a4f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
index 2db3165..9d9d759 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/FrozenSetVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.FrozenSetDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
index 93a43c1..e58ad4f 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetAsciiTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(new DataSet.AsciiDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
index a37e907..bdf6fe0 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetBigintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.BigintDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.BigintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
index 9d11fe3..460e70d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDateTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DateDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DateDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
index 95bccbf..ef495d9 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDecimalTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DecimalDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DecimalDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
index 05552dd..86a4fa5 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetDoubleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DoubleDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.DoubleDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
index 82861d3..6c29210 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFloatTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.FloatDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.FloatDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
index d632a40..da4aca7 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetFrozenCollectionTest.java
@@ -32,6 +32,6 @@
public static Collection<Object[]> generateParameters()
{
DataSet<Set<Integer>> frozen = new CollectionDataSet.FrozenSetDataSet<>(new DataSet.IntDataSet());
- return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(frozen));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(frozen));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
index b51394f..380ed4c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetInetTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
index 900fa71..add9799 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetIntTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.IntDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.IntDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
index a9020c1..fa14363 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetSmallintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.SmallintDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.SmallintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
index 45dd37b..36e2314 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTextTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TextDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TextDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
index d21b539..21272d1 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
index d97cbd9..e18ce5a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimestampTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimestampDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimestampDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
index b0edd06..65d5556 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTimeuuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeuuidDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TimeuuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
index f044717..1a701bf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetTinyintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TinyintDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.TinyintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
index 52a9790..b4e5541 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetUuidTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.UuidDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.UuidDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
index 07e5a6f..777ebbf 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/collections/sets/SetVarintTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateNumericParameters(new CollectionDataSet.SetDataSet<>(new DataSet.VarintDataSet()));
+ return generateParameters(new CollectionDataSet.SetDataSet<>(new DataSet.VarintDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
index d82e3bd..427ae5d 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleCollectionTest.java
@@ -31,7 +31,7 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new FrozenTupleDataSet(
+ return generateParameters(new FrozenTupleDataSet(
new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()),
new CollectionDataSet.SetDataSet<>(new DataSet.InetDataSet()),
new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet())
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
index ffae1fd..7733c3c 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new FrozenTupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+ return generateParameters(new FrozenTupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
index a66bb4c..98db788 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenTupleTupleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new FrozenTupleDataSet(new FrozenTupleDataSet(new DataSet.AsciiDataSet(), new DataSet.UuidDataSet()), new DataSet.AsciiDataSet()));
+ return generateParameters(new FrozenTupleDataSet(new FrozenTupleDataSet(new DataSet.AsciiDataSet(), new DataSet.UuidDataSet()), new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
index 80ba060..ddd25b2 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTCollectionTest.java
@@ -31,7 +31,7 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new FrozenUDTDataSet(
+ return generateParameters(new FrozenUDTDataSet(
new CollectionDataSet.ListDataSet<>(new DataSet.AsciiDataSet()),
new CollectionDataSet.MapDataSet<>(new DataSet.BigintDataSet())));
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
index 6271f00..2510e0b 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/FrozenUDTTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new FrozenUDTDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+ return generateParameters(new FrozenUDTDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
index e76f018..d692d8a 100644
--- a/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/cql/types/multicell/TupleTest.java
@@ -30,6 +30,6 @@
@Parameterized.Parameters(name = "dataset={0},wide={1},scenario={2}")
public static Collection<Object[]> generateParameters()
{
- return generateLiteralParameters(new TupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
+ return generateParameters(new TupleDataSet(new DataSet.BigintDataSet(), new DataSet.AsciiDataSet()));
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java b/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
index 51c7637..6806110 100644
--- a/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/disk/SelectiveIntersectionTest.java
@@ -17,18 +17,15 @@
*/
package org.apache.cassandra.index.sai.disk;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import com.datastax.driver.core.Session;
+import org.apache.cassandra.config.CassandraRelevantProperties;
import org.apache.cassandra.index.sai.SAITester;
import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
-import org.apache.cassandra.index.sai.iterators.KeyRangeIntersectionIterator;
import org.apache.cassandra.inject.Injections;
import static org.apache.cassandra.inject.InvokePointBuilder.newInvokePoint;
@@ -145,13 +142,8 @@
Assert.assertEquals(postingsReaderOpenCounter.get(), postingsReaderCloseCounter.get());
}
- private static void setLimits(final int selectivityLimit) throws Exception
+ private static void setLimits(final int selectivityLimit)
{
- Field selectivity = KeyRangeIntersectionIterator.class.getDeclaredField("INTERSECTION_CLAUSE_LIMIT");
- selectivity.setAccessible(true);
- Field modifiersField = Field.class.getDeclaredField("modifiers");
- modifiersField.setAccessible(true);
- modifiersField.setInt(selectivity, selectivity.getModifiers() & ~Modifier.FINAL);
- selectivity.set(null, selectivityLimit);
+ CassandraRelevantProperties.SAI_INTERSECTION_CLAUSE_LIMIT.setString(Integer.toString(selectivityLimit));
}
}
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 995a876..b6546c9 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
@@ -18,8 +18,6 @@
package org.apache.cassandra.index.sai.disk.format;
-import java.lang.reflect.Field;
-import java.lang.reflect.Modifier;
import java.net.URI;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -45,7 +43,6 @@
{
private final TemporaryFolder temporaryFolder = new TemporaryFolder();
private Descriptor descriptor;
- private Version latest;
@BeforeClass
public static void initialise()
@@ -58,21 +55,17 @@
{
temporaryFolder.create();
descriptor = Descriptor.fromFile(new File(temporaryFolder.newFolder().getAbsolutePath() + "/nb-1-big-Data.db"));
- latest = Version.LATEST;
}
@After
public void teardown() throws Throwable
{
- setLatestVersion(latest);
temporaryFolder.delete();
}
@Test
public void versionAAPerSSTableComponentIsParsedCorrectly() throws Throwable
{
- setLatestVersion(Version.AA);
-
createFileOnDisk("-SAI+aa+GroupComplete.db");
IndexDescriptor indexDescriptor = IndexDescriptor.create(descriptor, SAITester.EMPTY_COMPARATOR);
@@ -84,8 +77,6 @@
@Test
public void versionAAPerIndexComponentIsParsedCorrectly() throws Throwable
{
- setLatestVersion(Version.AA);
-
createFileOnDisk("-SAI+aa+test_index+ColumnComplete.db");
IndexDescriptor indexDescriptor = IndexDescriptor.create(descriptor, SAITester.EMPTY_COMPARATOR);
@@ -95,16 +86,6 @@
assertTrue(indexDescriptor.hasComponent(IndexComponent.COLUMN_COMPLETION_MARKER, indexContext));
}
- private void setLatestVersion(Version version) throws Throwable
- {
- Field latest = Version.class.getDeclaredField("LATEST");
- latest.setAccessible(true);
- Field modifiersField = Field.class.getDeclaredField("modifiers");
- modifiersField.setAccessible(true);
- modifiersField.setInt(latest, latest.getModifiers() & ~Modifier.FINAL);
- latest.set(null, version);
- }
-
private void createFileOnDisk(String filename) throws Throwable
{
Path path;
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java
new file mode 100644
index 0000000..b880a77
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/BalancedTreeIndexSearcherTest.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.NumberType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.BlockBalancedTreeIndexBuilder;
+import org.apache.cassandra.index.sai.disk.v1.segment.IndexSegmentSearcher;
+import org.apache.cassandra.index.sai.iterators.KeyRangeIterator;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class BalancedTreeIndexSearcherTest extends SAIRandomizedTester
+{
+ private static final short EQ_TEST_LOWER_BOUND_INCLUSIVE = 0;
+ private static final short EQ_TEST_UPPER_BOUND_EXCLUSIVE = 3;
+
+ private static final short RANGE_TEST_LOWER_BOUND_INCLUSIVE = 0;
+ private static final short RANGE_TEST_UPPER_BOUND_EXCLUSIVE = 10;
+
+ @Test
+ public void testRangeQueriesAgainstInt32Index() throws Exception
+ {
+ doTestRangeQueriesAgainstInt32Index();
+ }
+
+ private void doTestRangeQueriesAgainstInt32Index() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildInt32Searcher(newIndexDescriptor(), 0, 10);
+ testRangeQueries(indexSearcher, Int32Type.instance, Int32Type.instance, Integer::valueOf);
+ }
+
+ @Test
+ public void testEqQueriesAgainstInt32Index() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildInt32Searcher(newIndexDescriptor(),
+ EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+ testEqQueries(indexSearcher, Int32Type.instance, Int32Type.instance, Integer::valueOf);
+ }
+
+ @Test
+ public void testRangeQueriesAgainstLongIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildLongSearcher(newIndexDescriptor(), 0, 10);
+ testRangeQueries(indexSearcher, LongType.instance, Int32Type.instance, Long::valueOf);
+ }
+
+ @Test
+ public void testEqQueriesAgainstLongIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildLongSearcher(newIndexDescriptor(),
+ EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+ testEqQueries(indexSearcher, LongType.instance, Int32Type.instance, Long::valueOf);
+ }
+
+ @Test
+ public void testRangeQueriesAgainstShortIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildShortSearcher(newIndexDescriptor(), (short) 0, (short) 10);
+ testRangeQueries(indexSearcher, ShortType.instance, Int32Type.instance, Function.identity());
+ }
+
+ @Test
+ public void testEqQueriesAgainstShortIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildShortSearcher(newIndexDescriptor(),
+ EQ_TEST_LOWER_BOUND_INCLUSIVE, EQ_TEST_UPPER_BOUND_EXCLUSIVE);
+ testEqQueries(indexSearcher, ShortType.instance, Int32Type.instance, Function.identity());
+ }
+
+ @Test
+ public void testRangeQueriesAgainstDecimalIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildDecimalSearcher(newIndexDescriptor(),
+ BigDecimal.ZERO, BigDecimal.valueOf(10L));
+ testRangeQueries(indexSearcher, DecimalType.instance, DecimalType.instance, BigDecimal::valueOf,
+ getLongsOnInterval(21L, 70L));
+ }
+
+ private List<Long> getLongsOnInterval(long lowerInclusive, long upperInclusive)
+ {
+ return LongStream.range(lowerInclusive, upperInclusive + 1L).boxed().collect(Collectors.toList());
+ }
+
+ @Test
+ public void testEqQueriesAgainstDecimalIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildDecimalSearcher(newIndexDescriptor(),
+ BigDecimal.valueOf(EQ_TEST_LOWER_BOUND_INCLUSIVE), BigDecimal.valueOf(EQ_TEST_UPPER_BOUND_EXCLUSIVE));
+ testEqQueries(indexSearcher, DecimalType.instance, DecimalType.instance, BigDecimal::valueOf);
+ }
+
+
+ @Test
+ public void testEqQueriesAgainstBigIntegerIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildBigIntegerSearcher(newIndexDescriptor(),
+ BigInteger.valueOf(EQ_TEST_LOWER_BOUND_INCLUSIVE), BigInteger.valueOf(EQ_TEST_UPPER_BOUND_EXCLUSIVE));
+ testEqQueries(indexSearcher, IntegerType.instance, IntegerType.instance, BigInteger::valueOf);
+ }
+
+ @Test
+ public void testRangeQueriesAgainstBigIntegerIndex() throws Exception
+ {
+ IndexSegmentSearcher indexSearcher = BlockBalancedTreeIndexBuilder.buildBigIntegerSearcher(newIndexDescriptor(),
+ BigInteger.ZERO, BigInteger.valueOf(10L));
+ testRangeQueries(indexSearcher, IntegerType.instance, IntegerType.instance, BigInteger::valueOf);
+ }
+
+ private <T extends Number> void testEqQueries(final IndexSegmentSearcher indexSearcher,
+ final NumberType<T> rawType, final NumberType<?> encodedType,
+ final Function<Short, T> rawValueProducer) throws Exception
+ {
+ try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+ {{
+ operator = IndexOperator.EQ;
+ lower = upper = new Bound(rawType.decompose(rawValueProducer.apply(EQ_TEST_LOWER_BOUND_INCLUSIVE)), encodedType, true);
+ }}, mock(QueryContext.class)))
+ {
+ assertEquals(results.getMinimum(), results.getCurrent());
+ assertTrue(results.hasNext());
+
+ assertEquals(0L, results.next().token().getLongValue());
+ }
+
+ try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+ {{
+ operator = IndexOperator.EQ;
+ lower = upper = new Bound(rawType.decompose(rawValueProducer.apply(EQ_TEST_UPPER_BOUND_EXCLUSIVE)), encodedType, true);
+ }}, mock(QueryContext.class)))
+ {
+ assertFalse(results.hasNext());
+ indexSearcher.close();
+ }
+ }
+
+ private <T extends Number> void testRangeQueries(final IndexSegmentSearcher indexSearcher,
+ final NumberType<T> rawType, final NumberType<?> encodedType,
+ final Function<Short, T> rawValueProducer) throws Exception
+ {
+ List<Long> expectedTokenList = getLongsOnInterval(3L, 7L);
+ testRangeQueries(indexSearcher, rawType, encodedType, rawValueProducer, expectedTokenList);
+ }
+
+
+ private <T extends Number> void testRangeQueries(final IndexSegmentSearcher indexSearcher,
+ final NumberType<T> rawType, final NumberType<?> encodedType,
+ final Function<Short, T> rawValueProducer, List<Long> expectedTokenList) throws Exception
+ {
+ try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+ {{
+ operator = IndexOperator.RANGE;
+
+ lower = new Bound(rawType.decompose(rawValueProducer.apply((short)2)), encodedType, false);
+ upper = new Bound(rawType.decompose(rawValueProducer.apply((short)7)), encodedType, true);
+ }}, mock(QueryContext.class)))
+ {
+ assertEquals(results.getMinimum(), results.getCurrent());
+ assertTrue(results.hasNext());
+
+ List<Long> actualTokenList = Lists.newArrayList(Iterators.transform(results, key -> key.token().getLongValue()));
+ assertEquals(expectedTokenList, actualTokenList);
+ }
+
+ try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+ {{
+ operator = IndexOperator.RANGE;
+ lower = new Bound(rawType.decompose(rawValueProducer.apply(RANGE_TEST_UPPER_BOUND_EXCLUSIVE)), encodedType, true);
+ }}, mock(QueryContext.class)))
+ {
+ assertFalse(results.hasNext());
+ }
+
+ try (KeyRangeIterator results = indexSearcher.search(new Expression(SAITester.createIndexContext("meh", rawType))
+ {{
+ operator = IndexOperator.RANGE;
+ upper = new Bound(rawType.decompose(rawValueProducer.apply(RANGE_TEST_LOWER_BOUND_INCLUSIVE)), encodedType, false);
+ }}, mock(QueryContext.class)))
+ {
+ assertFalse(results.hasNext());
+ indexSearcher.close();
+ }
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
index b317f4f..2512343 100644
--- a/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/TermsScanner.java
@@ -30,6 +30,7 @@
import org.apache.cassandra.io.util.FileHandle;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.Throwables;
import org.apache.cassandra.utils.bytecomparable.ByteComparable;
import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
import org.apache.lucene.store.IndexInput;
@@ -51,11 +52,18 @@
@Override
@SuppressWarnings({"resource", "RedundantSuppression"})
- public PostingList postings() throws IOException
+ public PostingList postings()
{
assert entry != null;
final IndexInput input = IndexFileUtils.instance.openInput(postingsFile);
- return new ScanningPostingsReader(input, new PostingsReader.BlocksSummary(input, entry.right));
+ try
+ {
+ return new ScanningPostingsReader(input, new PostingsReader.BlocksSummary(input, entry.right));
+ }
+ catch (IOException e)
+ {
+ throw Throwables.unchecked(e);
+ }
}
@Override
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java
new file mode 100644
index 0000000..b69e515
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeIndexBuilder.java
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+import java.util.stream.Stream;
+
+import org.junit.Assert;
+
+import com.carrotsearch.hppc.LongArrayList;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.ShortType;
+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.PrimaryKeyMap;
+import org.apache.cassandra.index.sai.disk.format.IndexDescriptor;
+import org.apache.cassandra.index.sai.disk.v1.segment.NumericIndexSegmentSearcher;
+import org.apache.cassandra.index.sai.disk.v1.PerColumnIndexFiles;
+import org.apache.cassandra.index.sai.disk.v1.segment.IndexSegmentSearcher;
+import org.apache.cassandra.index.sai.disk.v1.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.memory.MemtableTermsIterator;
+import org.apache.cassandra.index.sai.utils.PrimaryKey;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.AbstractGuavaIterator;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeIndexBuilder
+{
+ public static final PrimaryKeyMap TEST_PRIMARY_KEY_MAP = new PrimaryKeyMap()
+ {
+ private final PrimaryKey.Factory primaryKeyFactory = new PrimaryKey.Factory(null);
+
+ @Override
+ public PrimaryKey primaryKeyFromRowId(long sstableRowId)
+ {
+ return primaryKeyFactory.createTokenOnly(new Murmur3Partitioner.LongToken(sstableRowId));
+ }
+
+ @Override
+ public long rowIdFromPrimaryKey(PrimaryKey key)
+ {
+ return key.token().getLongValue();
+ }
+ };
+ public static final PrimaryKeyMap.Factory TEST_PRIMARY_KEY_MAP_FACTORY = () -> TEST_PRIMARY_KEY_MAP;
+
+
+ private static final BigDecimal ONE_TENTH = BigDecimal.valueOf(1, 1);
+
+ private final IndexDescriptor indexDescriptor;
+ private final AbstractType<?> type;
+ private final AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> terms;
+ private final int size;
+ private final int minSegmentRowId;
+ private final int maxSegmentRowId;
+
+ public BlockBalancedTreeIndexBuilder(IndexDescriptor indexDescriptor,
+ AbstractType<?> type,
+ AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> terms,
+ int size,
+ int minSegmentRowId,
+ int maxSegmentRowId)
+ {
+ this.indexDescriptor = indexDescriptor;
+ this.type = type;
+ this.terms = terms;
+ this.size = size;
+ this.minSegmentRowId = minSegmentRowId;
+ this.maxSegmentRowId = maxSegmentRowId;
+ }
+
+ NumericIndexSegmentSearcher flushAndOpen() throws IOException
+ {
+ final TermsIterator termEnum = new MemtableTermsIterator(null, null, terms);
+ final SegmentMetadata metadata;
+
+ IndexContext columnContext = SAITester.createIndexContext("test", Int32Type.instance);
+ NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ columnContext,
+ TypeUtil.fixedSizeOf(type),
+ maxSegmentRowId);
+ final SegmentMetadata.ComponentMetadataMap indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(termEnum, type));
+ metadata = new SegmentMetadata(0,
+ size,
+ minSegmentRowId,
+ maxSegmentRowId,
+ // min/max is unused for now
+ SAITester.TEST_FACTORY.createTokenOnly(Murmur3Partitioner.instance.decorateKey(UTF8Type.instance.fromString("a")).getToken()),
+ SAITester.TEST_FACTORY.createTokenOnly(Murmur3Partitioner.instance.decorateKey(UTF8Type.instance.fromString("b")).getToken()),
+ UTF8Type.instance.fromString("c"),
+ UTF8Type.instance.fromString("d"),
+ indexMetas);
+
+ try (PerColumnIndexFiles indexFiles = new PerColumnIndexFiles(indexDescriptor, SAITester.createIndexContext("test", Int32Type.instance)))
+ {
+ IndexSegmentSearcher searcher = IndexSegmentSearcher.open(TEST_PRIMARY_KEY_MAP_FACTORY, indexFiles, metadata, columnContext);
+ assertThat(searcher, is(instanceOf(NumericIndexSegmentSearcher.class)));
+ return (NumericIndexSegmentSearcher) searcher;
+ }
+ }
+
+ /**
+ * Returns a k-d tree index where:
+ * 1. term values have 32b
+ * 2. term value is equal to {@code startTermInclusive} + row id;
+ * 3. tokens and offsets are equal to row id;
+ */
+ public static IndexSegmentSearcher buildInt32Searcher(IndexDescriptor indexDescriptor, int startTermInclusive, int endTermExclusive)
+ throws IOException
+ {
+ final int size = endTermExclusive - startTermInclusive;
+ Assert.assertTrue(size > 0);
+ BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+ Int32Type.instance,
+ singleOrd(int32Range(startTermInclusive, endTermExclusive), Int32Type.instance, startTermInclusive, size),
+ size,
+ startTermInclusive,
+ endTermExclusive);
+ return indexBuilder.flushAndOpen();
+ }
+
+ public static IndexSegmentSearcher buildDecimalSearcher(IndexDescriptor indexDescriptor, BigDecimal startTermInclusive, BigDecimal endTermExclusive)
+ throws IOException
+ {
+ BigDecimal bigDifference = endTermExclusive.subtract(startTermInclusive);
+ int size = bigDifference.intValueExact() * 10;
+ Assert.assertTrue(size > 0);
+ BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+ DecimalType.instance,
+ singleOrd(decimalRange(startTermInclusive, endTermExclusive), DecimalType.instance, startTermInclusive.intValueExact() * 10, size),
+ size,
+ startTermInclusive.intValueExact() * 10,
+ endTermExclusive.intValueExact() * 10);
+ return indexBuilder.flushAndOpen();
+ }
+
+ public static IndexSegmentSearcher buildBigIntegerSearcher(IndexDescriptor indexDescriptor, BigInteger startTermInclusive, BigInteger endTermExclusive)
+ throws IOException
+ {
+ BigInteger bigDifference = endTermExclusive.subtract(startTermInclusive);
+ int size = bigDifference.intValueExact();
+ Assert.assertTrue(size > 0);
+ BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+ IntegerType.instance,
+ singleOrd(bigIntegerRange(startTermInclusive, endTermExclusive), IntegerType.instance, startTermInclusive.intValueExact(), size),
+ size,
+ startTermInclusive.intValueExact(),
+ endTermExclusive.intValueExact());
+ return indexBuilder.flushAndOpen();
+ }
+
+ /**
+ * Returns a k-d tree index where:
+ * 1. term values have 64b
+ * 2. term value is equal to {@code startTermInclusive} + row id;
+ * 3. tokens and offsets are equal to row id;
+ */
+ public static IndexSegmentSearcher buildLongSearcher(IndexDescriptor indexDescriptor, long startTermInclusive, long endTermExclusive)
+ throws IOException
+ {
+ final long size = endTermExclusive - startTermInclusive;
+ Assert.assertTrue(size > 0);
+ BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+ LongType.instance,
+ singleOrd(longRange(startTermInclusive, endTermExclusive), LongType.instance, Math.toIntExact(startTermInclusive), Math.toIntExact(size)),
+ Math.toIntExact(size),
+ Math.toIntExact(startTermInclusive),
+ Math.toIntExact(endTermExclusive));
+ return indexBuilder.flushAndOpen();
+ }
+
+ /**
+ * Returns a k-d tree index where:
+ * 1. term values have 16b
+ * 2. term value is equal to {@code startTermInclusive} + row id;
+ * 3. tokens and offsets are equal to row id;
+ */
+ public static IndexSegmentSearcher buildShortSearcher(IndexDescriptor indexDescriptor, short startTermInclusive, short endTermExclusive)
+ throws IOException
+ {
+ final int size = endTermExclusive - startTermInclusive;
+ Assert.assertTrue(size > 0);
+ BlockBalancedTreeIndexBuilder indexBuilder = new BlockBalancedTreeIndexBuilder(indexDescriptor,
+ ShortType.instance,
+ singleOrd(shortRange(startTermInclusive, endTermExclusive), ShortType.instance, startTermInclusive, size),
+ size,
+ startTermInclusive,
+ endTermExclusive);
+ return indexBuilder.flushAndOpen();
+ }
+
+ /**
+ * Returns inverted index where each posting list contains exactly one element equal to the terms ordinal number +
+ * given offset.
+ */
+ public static AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> singleOrd(Iterator<ByteBuffer> terms, AbstractType<?> type, int segmentRowIdOffset, int size)
+ {
+ return new AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>>()
+ {
+ private long currentTerm = 0;
+ private int currentSegmentRowId = segmentRowIdOffset;
+
+ @Override
+ protected Pair<ByteComparable, LongArrayList> computeNext()
+ {
+ if (currentTerm++ >= size)
+ {
+ return endOfData();
+ }
+
+ LongArrayList postings = new LongArrayList();
+ postings.add(currentSegmentRowId++);
+ assertTrue(terms.hasNext());
+
+ final ByteSource encoded = TypeUtil.asComparableBytes(terms.next(), type, ByteComparable.Version.OSS50);
+ return Pair.create(v -> encoded, postings);
+ }
+ };
+ }
+
+ /**
+ * Returns sequential ordered encoded ints from {@code startInclusive} (inclusive) to {@code endExclusive}
+ * (exclusive) by an incremental step of {@code 1}.
+ */
+ public static Iterator<ByteBuffer> int32Range(int startInclusive, int endExclusive)
+ {
+ return IntStream.range(startInclusive, endExclusive)
+ .mapToObj(Int32Type.instance::decompose)
+ .collect(Collectors.toList())
+ .iterator();
+ }
+
+ /**
+ * Returns sequential ordered encoded longs from {@code startInclusive} (inclusive) to {@code endExclusive}
+ * (exclusive) by an incremental step of {@code 1}.
+ */
+ public static Iterator<ByteBuffer> longRange(long startInclusive, long endExclusive)
+ {
+ return LongStream.range(startInclusive, endExclusive)
+ .mapToObj(LongType.instance::decompose)
+ .collect(Collectors.toList())
+ .iterator();
+ }
+
+ public static Iterator<ByteBuffer> decimalRange(final BigDecimal startInclusive, final BigDecimal endExclusive)
+ {
+ int n = endExclusive.subtract(startInclusive).intValueExact() * 10;
+ final Supplier<BigDecimal> generator = new Supplier<BigDecimal>() {
+ BigDecimal current = startInclusive;
+
+ @Override
+ public BigDecimal get() {
+ BigDecimal result = current;
+ current = current.add(ONE_TENTH);
+ return result;
+ }
+ };
+ return Stream.generate(generator)
+ .limit(n)
+ .map(bd -> TypeUtil.asIndexBytes(DecimalType.instance.decompose(bd), DecimalType.instance))
+ .collect(Collectors.toList())
+ .iterator();
+ }
+
+ public static Iterator<ByteBuffer> bigIntegerRange(final BigInteger startInclusive, final BigInteger endExclusive)
+ {
+ int n = endExclusive.subtract(startInclusive).intValueExact();
+ final Supplier<BigInteger> generator = new Supplier<BigInteger>() {
+ BigInteger current = startInclusive;
+
+ @Override
+ public BigInteger get() {
+ BigInteger result = current;
+ current = current.add(BigInteger.ONE);
+ return result;
+ }
+ };
+ return Stream.generate(generator)
+ .limit(n)
+ .map(bd -> TypeUtil.asIndexBytes(IntegerType.instance.decompose(bd), IntegerType.instance))
+ .collect(Collectors.toList())
+ .iterator();
+ }
+
+
+ /**
+ * Returns sequential ordered encoded shorts from {@code startInclusive} (inclusive) to {@code endExclusive}
+ * (exclusive) by an incremental step of {@code 1}.
+ */
+ public static Iterator<ByteBuffer> shortRange(short startInclusive, short endExclusive)
+ {
+ return IntStream.range(startInclusive, endExclusive)
+ .mapToObj(i -> ShortType.instance.decompose((short) i))
+ .collect(Collectors.toList())
+ .iterator();
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java
new file mode 100644
index 0000000..6f1daed
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreePostingsWriterTest.java
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.agrona.collections.IntArrayList;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.ArrayPostingList;
+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.io.IndexOutputWriter;
+import org.apache.cassandra.index.sai.disk.v1.postings.PostingsReader;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedLongValues;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class BlockBalancedTreePostingsWriterTest extends SAIRandomizedTester
+{
+ private IndexDescriptor indexDescriptor;
+ private IndexContext indexContext;
+
+ @Before
+ public void setup() throws Throwable
+ {
+ indexDescriptor = newIndexDescriptor();
+ indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+ }
+
+ @Test
+ public void shouldWritePostingsForEligibleNodes() throws Exception
+ {
+ List<PackedLongValues> leaves =
+ Arrays.asList(postings(1, 5, 7), postings(3, 4, 6), postings(2, 8, 10), postings(11, 12, 13));
+
+ setBDKPostingsWriterSizing(1, 2);
+ BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+ // should build postings for nodes 2 & 3 (lvl 2) and 8, 10, 12, 14 (lvl 4)
+ writer.onLeaf(64, 1, pathToRoot(1, 2, 4, 8, 16));
+ writer.onLeaf(80, 2, pathToRoot(1, 2, 5, 10, 20));
+ writer.onLeaf(96, 3, pathToRoot(1, 3, 6, 12, 24));
+ writer.onLeaf(112, 4, pathToRoot(1, 3, 7, 14, 28));
+
+ long fp;
+ try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+ {
+ fp = writer.finish(output, leaves, indexContext);
+ }
+
+ BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+ assertEquals(10, postingsIndex.size());
+
+ // Internal postings...
+ assertTrue(postingsIndex.exists(2));
+ assertTrue(postingsIndex.exists(3));
+ assertTrue(postingsIndex.exists(8));
+ assertTrue(postingsIndex.exists(10));
+ assertTrue(postingsIndex.exists(12));
+ assertTrue(postingsIndex.exists(14));
+
+ assertPostingReaderEquals(postingsIndex, 2, 1, 3, 4, 5, 6, 7);
+ assertPostingReaderEquals(postingsIndex, 3, 2, 8, 10, 11, 12, 13);
+ assertPostingReaderEquals(postingsIndex, 8, 1, 5, 7);
+ assertPostingReaderEquals(postingsIndex, 10, 3, 4, 6);
+ assertPostingReaderEquals(postingsIndex, 12, 2, 8, 10);
+ assertPostingReaderEquals(postingsIndex, 14, 11, 12, 13);
+
+ // Leaf postings...
+ assertTrue(postingsIndex.exists(64));
+ assertTrue(postingsIndex.exists(80));
+ assertTrue(postingsIndex.exists(96));
+ assertTrue(postingsIndex.exists(112));
+
+ assertPostingReaderEquals(postingsIndex, 64, 1, 5, 7);
+ assertPostingReaderEquals(postingsIndex, 80, 3, 4, 6);
+ assertPostingReaderEquals(postingsIndex, 96, 2, 8, 10);
+ assertPostingReaderEquals(postingsIndex, 112, 11, 12, 13);
+ }
+
+ @Test
+ public void shouldSkipPostingListWhenSamplingMisses() throws Exception
+ {
+ List<PackedLongValues> leaves = Collections.singletonList(postings(1, 2, 3));
+
+ setBDKPostingsWriterSizing(1, 5);
+ BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+ // The tree is too short to have any internal posting lists.
+ writer.onLeaf(16, 1, pathToRoot(1, 2, 4, 8));
+
+ long fp;
+ try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+ {
+ fp = writer.finish(output, leaves, indexContext);
+ }
+
+ // There is only a single posting list...the leaf posting list.
+ BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+ assertEquals(1, postingsIndex.size());
+ }
+
+ @Test
+ public void shouldSkipPostingListWhenTooFewLeaves() throws Exception
+ {
+ List<PackedLongValues> leaves = Collections.singletonList(postings(1, 2, 3));
+
+ setBDKPostingsWriterSizing(2, 2);
+ BlockBalancedTreePostingsWriter writer = new BlockBalancedTreePostingsWriter();
+
+ // The tree is too short to have any internal posting lists.
+ writer.onLeaf(16, 1, pathToRoot(1, 2, 4, 8));
+
+ long fp;
+ try (IndexOutputWriter output = indexDescriptor.openPerIndexOutput(IndexComponent.POSTING_LISTS, indexContext))
+ {
+ fp = writer.finish(output, leaves, indexContext);
+ }
+
+ // There is only a single posting list...the leaf posting list.
+ BlockBalancedTreePostingsIndex postingsIndex = new BlockBalancedTreePostingsIndex(indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext), fp);
+ assertEquals(1, postingsIndex.size());
+ }
+
+ private void assertPostingReaderEquals(BlockBalancedTreePostingsIndex postingsIndex, int nodeID, long... postings) throws IOException
+ {
+ assertPostingReaderEquals(indexDescriptor.openPerIndexInput(IndexComponent.POSTING_LISTS, indexContext),
+ postingsIndex.getPostingsFilePointer(nodeID),
+ new ArrayPostingList(postings));
+ }
+
+ private void assertPostingReaderEquals(IndexInput input, long offset, PostingList expected) throws IOException
+ {
+ try (PostingsReader reader = new PostingsReader(input, offset, mock(QueryEventListener.PostingListEventListener.class)))
+ {
+ assertPostingListEquals(expected, reader);
+ }
+ }
+
+ private PackedLongValues postings(int... postings)
+ {
+ final PackedLongValues.Builder builder = PackedLongValues.deltaPackedBuilder(PackedInts.COMPACT);
+ for (int posting : postings)
+ {
+ builder.add(posting);
+ }
+ return builder.build();
+ }
+
+ private IntArrayList pathToRoot(int... nodes)
+ {
+ final IntArrayList path = new IntArrayList();
+ for (int node : nodes)
+ {
+ path.add(node);
+ }
+ return path;
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java
new file mode 100644
index 0000000..a055236
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeQueriesTest.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+
+import static org.apache.lucene.index.PointValues.Relation.CELL_CROSSES_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_INSIDE_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_OUTSIDE_QUERY;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeQueriesTest extends SAIRandomizedTester
+{
+ @Test
+ public void testMatchesAll()
+ {
+ Expression expression = new Expression(SAITester.createIndexContext("meh", Int32Type.instance));
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ for (int visit = 0; visit < between(100, 1000); visit++)
+ assertTrue(query.contains(toSortableBytes(nextInt(Integer.MAX_VALUE))));
+
+ for (int compare = 0; compare < between(100, 1000); compare++)
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(between(0, Integer.MAX_VALUE/2)),
+ toSortableBytes(between(Integer.MAX_VALUE/2, Integer.MAX_VALUE))));
+ }
+
+ @Test
+ public void testInclusiveLowerBound()
+ {
+ int lowerBound = between(-10, 10);
+ Expression expression = buildExpression(Operator.GTE, lowerBound);
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertTrue(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ }
+
+ @Test
+ public void testExclusiveLowerBound()
+ {
+ int lowerBound = between(-10, 10);
+ Expression expression = buildExpression(Operator.GT, lowerBound);
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertFalse(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(lowerBound + 2)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+ }
+
+ @Test
+ public void testInclusiveUpperBound()
+ {
+ int upperBound = between(-10, 10);
+ Expression expression = buildExpression(Operator.LTE, upperBound);
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound)));
+ assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+ }
+
+ @Test
+ public void testExclusiveUpperBound()
+ {
+ int upper = between(-10, 10);
+ Expression expression = buildExpression(Operator.LT, upper);
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertTrue(query.contains(toSortableBytes(upper - 1)));
+ assertFalse(query.contains(toSortableBytes(upper)));
+ assertFalse(query.contains(toSortableBytes(upper + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upper), toSortableBytes(upper + 1)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(upper - 2), toSortableBytes(upper - 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upper - 1), toSortableBytes(upper)));
+ }
+
+ @Test
+ public void testInclusiveLowerAndUpperBound()
+ {
+ int lowerBound = between(-15, 15);
+ int upperBound = lowerBound + 5;
+ Expression expression = buildExpression(Operator.GTE, lowerBound).add(Operator.LTE, Int32Type.instance.decompose(upperBound));
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertTrue(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound)));
+ assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(upperBound)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+ }
+
+ @Test
+ public void testExclusiveLowerAndUpperBound()
+ {
+ int lowerBound = between(-15, 15);
+ int upperBound = lowerBound + 5;
+ Expression expression = buildExpression(Operator.GT, lowerBound).add(Operator.LT, Int32Type.instance.decompose(upperBound));
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertFalse(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+ assertFalse(query.contains(toSortableBytes(upperBound)));
+ assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(upperBound - 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+ }
+
+ @Test
+ public void testExclusiveLowerAndInclusiveUpperBound()
+ {
+ int lowerBound = between(-15, 15);
+ int upperBound = lowerBound + 5;
+ Expression expression = buildExpression(Operator.GT, lowerBound).add(Operator.LTE, Int32Type.instance.decompose(upperBound));
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertFalse(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound)));
+ assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(lowerBound + 1)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound + 1), toSortableBytes(upperBound)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound + 1), toSortableBytes(upperBound + 2)));
+ }
+
+ @Test
+ public void testInclusiveLowerAndExclusiveUpperBound()
+ {
+ int lowerBound = between(-15, 15);
+ int upperBound = lowerBound + 5;
+ Expression expression = buildExpression(Operator.GTE, lowerBound).add(Operator.LT, Int32Type.instance.decompose(upperBound));
+ BlockBalancedTreeReader.IntersectVisitor query = BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4);
+
+ assertFalse(query.contains(toSortableBytes(lowerBound - 1)));
+ assertTrue(query.contains(toSortableBytes(lowerBound)));
+ assertTrue(query.contains(toSortableBytes(lowerBound + 1)));
+ assertTrue(query.contains(toSortableBytes(upperBound - 1)));
+ assertFalse(query.contains(toSortableBytes(upperBound)));
+ assertFalse(query.contains(toSortableBytes(upperBound + 1)));
+
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(lowerBound - 2), toSortableBytes(lowerBound - 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(lowerBound - 1), toSortableBytes(lowerBound)));
+ assertEquals(CELL_INSIDE_QUERY, query.compare(toSortableBytes(lowerBound), toSortableBytes(upperBound - 1)));
+ assertEquals(CELL_CROSSES_QUERY, query.compare(toSortableBytes(upperBound - 1), toSortableBytes(upperBound)));
+ assertEquals(CELL_OUTSIDE_QUERY, query.compare(toSortableBytes(upperBound), toSortableBytes(upperBound + 1)));
+ }
+
+ private byte[] toSortableBytes(int value)
+ {
+ byte[] buffer = new byte[4];
+ ByteSource source = Int32Type.instance.asComparableBytes(Int32Type.instance.decompose(value), ByteComparable.Version.OSS50);
+ ByteSourceInverse.copyBytes(source, buffer);
+ return buffer;
+ }
+
+ private Expression buildExpression(Operator op, int value)
+ {
+ Expression expression = new Expression(SAITester.createIndexContext("meh", Int32Type.instance));
+ expression.add(op, Int32Type.instance.decompose(value));
+ return expression;
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java
new file mode 100644
index 0000000..da5faa9
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeRamBufferTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.StreamSupport;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+
+import static org.junit.Assert.assertEquals;
+
+public class BlockBalancedTreeRamBufferTest extends SAIRandomizedTester
+{
+ @Test
+ public void shouldReturnValuesInSortedValue()
+ {
+ int numRows = nextInt(100, 1000);
+
+ // Generate a random unsorted list of integers
+ List<Integer> values = IntStream.generate(() -> nextInt(0, 1000))
+ .distinct()
+ .limit(numRows)
+ .boxed()
+ .collect(Collectors.toList());
+
+ BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+ byte[] scratch = new byte[Integer.BYTES];
+ values.forEach(v -> {
+ TypeUtil.toComparableBytes(Int32Type.instance.decompose(v), Int32Type.instance, scratch);
+ buffer.add(0, scratch);
+ });
+
+ Iterable<Pair<byte[], PostingList>> iterable = buffer::iterator;
+
+ List<Integer> result = StreamSupport.stream(iterable.spliterator(), false).mapToInt(pair -> unpackValue(pair.left)).boxed().collect(Collectors.toList());
+
+ Collections.sort(values);
+
+ assertEquals(values, result);
+ }
+
+ private static int unpackValue(byte[] value)
+ {
+ return Int32Type.instance.compose(Int32Type.instance.fromComparableBytes(ByteSource.peekable(ByteSource.fixedLength(value)),
+ ByteComparable.Version.OSS50));
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java
new file mode 100644
index 0000000..984a064
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeReaderTest.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.Operator;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+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.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.plan.Expression;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.apache.lucene.index.PointValues.Relation.CELL_CROSSES_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_INSIDE_QUERY;
+import static org.apache.lucene.index.PointValues.Relation.CELL_OUTSIDE_QUERY;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class BlockBalancedTreeReaderTest extends SAIRandomizedTester
+{
+ private final BlockBalancedTreeReader.IntersectVisitor NONE_MATCH = new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ return false;
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ return CELL_OUTSIDE_QUERY;
+ }
+ };
+
+ private final BlockBalancedTreeReader.IntersectVisitor ALL_MATCH = new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ return true;
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ return CELL_INSIDE_QUERY;
+ }
+ };
+
+ private IndexDescriptor indexDescriptor;
+ private IndexContext indexContext;
+
+ @Before
+ public void setup() throws Throwable
+ {
+ indexDescriptor = newIndexDescriptor();
+ indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+ }
+
+ @Test
+ public void testFilteringIntersection() throws Exception
+ {
+ int numRows = 1000;
+
+ final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+ byte[] scratch = new byte[4];
+ for (int docID = 0; docID < numRows; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ final BlockBalancedTreeReader reader = finishAndOpenReader(4, buffer);
+
+ Expression expression = new Expression(indexContext);
+ expression.add(Operator.GT, Int32Type.instance.decompose(444));
+ expression.add(Operator.LT, Int32Type.instance.decompose(555));
+ PostingList intersection = performIntersection(reader, BlockBalancedTreeQueries.balancedTreeQueryFrom(expression, 4));
+ assertNotNull(intersection);
+ assertEquals(110, intersection.size());
+ for (long posting = 445; posting < 555; posting++)
+ assertEquals(posting, intersection.nextPosting());
+ }
+
+ @Test
+ public void testAdvance() throws Exception
+ {
+ final int numRows = between(1000, 2000);
+ final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+ byte[] scratch = new byte[4];
+ for (int docID = 0; docID < numRows; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ final BlockBalancedTreeReader reader = finishAndOpenReader(2, buffer);
+
+ PostingList intersection = performIntersection(reader, NONE_MATCH);
+ assertNull(intersection);
+
+ intersection = performIntersection(reader, ALL_MATCH);
+ assertEquals(numRows, intersection.size());
+ assertEquals(100, intersection.advance(100));
+ assertEquals(200, intersection.advance(200));
+ assertEquals(300, intersection.advance(300));
+ assertEquals(400, intersection.advance(400));
+ assertEquals(401, intersection.advance(401));
+ long expectedRowID = 402;
+ for (long id = intersection.nextPosting(); expectedRowID < 500; id = intersection.nextPosting())
+ {
+ assertEquals(expectedRowID++, id);
+ }
+ assertEquals(PostingList.END_OF_STREAM, intersection.advance(numRows + 1));
+
+ intersection.close();
+ }
+
+ @Test
+ public void testSameValuesInLeaf() throws Exception
+ {
+ // While a bit synthetic this test is designed to test that the
+ // BlockBalancedTreeReader.FilteringIntersection.buildPostingsFilterForSingleValueLeaf
+ // method is exercised in a test. To do this we need to ensure that
+ // we have at least one leaf that has all the same value and that
+ // all of that leaf is requested in a query.
+ final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+ byte[] scratch = new byte[4];
+
+ for (int docID = 0; docID < 10; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ for (int docID = 10; docID < 20; docID++)
+ {
+ NumericUtils.intToSortableBytes(10, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ for (int docID = 20; docID < 30; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ final BlockBalancedTreeReader reader = finishAndOpenReader(5, buffer);
+
+ PostingList postingList = performIntersection(reader, buildQuery(8, 15));
+
+ assertEquals(8, postingList.nextPosting());
+ assertEquals(9, postingList.nextPosting());
+ assertEquals(10, postingList.nextPosting());
+ assertEquals(11, postingList.nextPosting());
+ assertEquals(12, postingList.nextPosting());
+ assertEquals(13, postingList.nextPosting());
+ assertEquals(14, postingList.nextPosting());
+ assertEquals(15, postingList.nextPosting());
+ assertEquals(16, postingList.nextPosting());
+ assertEquals(17, postingList.nextPosting());
+ assertEquals(18, postingList.nextPosting());
+ assertEquals(19, postingList.nextPosting());
+ assertEquals(PostingList.END_OF_STREAM, postingList.nextPosting());
+ }
+
+ @Test
+ public void testResourcesReleaseWhenQueryDoesntMatchAnything() throws Exception
+ {
+ final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+ byte[] scratch = new byte[4];
+ for (int docID = 0; docID < 1000; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+ // add a gap between 1000 and 1100
+ for (int docID = 1000; docID < 2000; docID++)
+ {
+ NumericUtils.intToSortableBytes(docID + 100, scratch, 0);
+ buffer.add(docID, scratch);
+ }
+
+ final BlockBalancedTreeReader reader = finishAndOpenReader(50, buffer);
+
+ final PostingList intersection = performIntersection(reader, buildQuery(1017, 1096));
+ assertNull(intersection);
+ }
+
+ private PostingList performIntersection(BlockBalancedTreeReader reader, BlockBalancedTreeReader.IntersectVisitor visitor)
+ {
+ QueryEventListener.BalancedTreeEventListener balancedTreeEventListener = mock(QueryEventListener.BalancedTreeEventListener.class);
+ when(balancedTreeEventListener.postingListEventListener()).thenReturn(mock(QueryEventListener.PostingListEventListener.class));
+ return reader.intersect(visitor, balancedTreeEventListener, mock(QueryContext.class));
+ }
+
+ private BlockBalancedTreeReader.IntersectVisitor buildQuery(int queryMin, int queryMax)
+ {
+ return new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ int x = NumericUtils.sortableBytesToInt(packedValue, 0);
+ return x >= queryMin && x <= queryMax;
+ }
+
+ @Override
+ public Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ int min = NumericUtils.sortableBytesToInt(minPackedValue, 0);
+ int max = NumericUtils.sortableBytesToInt(maxPackedValue, 0);
+ assert max >= min;
+
+ if (max < queryMin || min > queryMax)
+ {
+ return Relation.CELL_OUTSIDE_QUERY;
+ }
+ else if (min >= queryMin && max <= queryMax)
+ {
+ return CELL_INSIDE_QUERY;
+ }
+ else
+ {
+ return CELL_CROSSES_QUERY;
+ }
+ }
+ };
+ }
+
+ private BlockBalancedTreeReader finishAndOpenReader(int maxPointsPerLeaf, BlockBalancedTreeRamBuffer buffer) throws Exception
+ {
+ setBDKPostingsWriterSizing(8, 2);
+ final NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ indexContext,
+ maxPointsPerLeaf,
+ Integer.BYTES,
+ Math.toIntExact(buffer.numRows()));
+
+ final SegmentMetadata.ComponentMetadataMap metadata = writer.writeCompleteSegment(buffer.iterator());
+ final long treePosition = metadata.get(IndexComponent.BALANCED_TREE).root;
+ assertThat(treePosition, is(greaterThan(0L)));
+ final long postingsPosition = metadata.get(IndexComponent.POSTING_LISTS).root;
+ assertThat(postingsPosition, is(greaterThan(0L)));
+
+ FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+ FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+ return new BlockBalancedTreeReader(indexContext,
+ treeHandle,
+ treePosition,
+ treePostingsHandle,
+ postingsPosition);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java
new file mode 100644
index 0000000..710da74
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/BlockBalancedTreeTest.java
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.util.function.IntFunction;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class BlockBalancedTreeTest extends SAIRandomizedTester
+{
+ private ByteBuffersDataOutput dataOutput;
+
+ @Before
+ public void setupDataOutput()
+ {
+ dataOutput = new ByteBuffersDataOutput();
+ }
+
+ @Test
+ public void testEmptyTree() throws Exception
+ {
+ long treeFilePointer = writeBalancedTree(0, 100, rowID -> rowID);
+
+ assertEquals(-1, treeFilePointer);
+ }
+
+ @Test
+ public void testSingleLeaf() throws Exception
+ {
+ BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(100, 100, rowID -> rowID);
+
+ assertEquals(1, state.numLeaves);
+ assertEquals(1, state.treeDepth);
+ assertEquals(100, state.valueCount);
+ assertTrue(state.atLeafNode());
+
+ recursiveAssertTraversal(state, -1);
+
+ assertEquals(state.treeDepth, state.maxLevel + 1);
+ }
+
+ @Test
+ public void testTreeWithSameValue() throws Exception
+ {
+ BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(100, 4, rowID -> 1);
+
+ recursiveAssertTraversal(state, -1);
+
+ assertEquals(state.treeDepth, state.maxLevel + 1);
+ }
+
+ @Test
+ public void testTreeDepthNeverMoreThanNumberOfLeaves() throws Exception
+ {
+ int leafSize = 4;
+ for (int numLeaves = 1; numLeaves < 1000; numLeaves++)
+ {
+ int numRows = leafSize * numLeaves;
+
+ BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(numRows, leafSize, rowID -> rowID);
+
+ assertEquals(numLeaves, state.numLeaves);
+ assertTrue(state.treeDepth <= state.numLeaves);
+
+ recursiveAssertTraversal(state, -1);
+
+ assertEquals(state.treeDepth, state.maxLevel + 1);
+ }
+ }
+
+ @Test
+ public void randomisedTreeTest() throws Exception
+ {
+ int loops = nextInt(10, 1000);
+
+ for (int loop = 0; loop < loops; loop++)
+ {
+ int leafSize = nextInt(2, 512);
+ int numRows = nextInt(1000, 10000);
+
+ BlockBalancedTreeWalker.TraversalState state = generateBalancedTree(numRows, leafSize, rowID -> nextInt(0, numRows / 2));
+
+ recursiveAssertTraversal(state, -1);
+
+ assertEquals(state.treeDepth, state.maxLevel + 1);
+ }
+ }
+
+ private long recursiveAssertTraversal(BlockBalancedTreeWalker.TraversalState state, long lastLeafBlockFP)
+ {
+ if (state.atLeafNode())
+ {
+ assertTrue(state.nodeExists());
+ assertTrue(state.getLeafBlockFP() > lastLeafBlockFP);
+ return state.getLeafBlockFP();
+ }
+ else
+ {
+ state.pushLeft();
+ lastLeafBlockFP = recursiveAssertTraversal(state, lastLeafBlockFP);
+ state.pop();
+
+ state.pushRight();
+ lastLeafBlockFP = recursiveAssertTraversal(state, lastLeafBlockFP);
+ state.pop();
+
+ return lastLeafBlockFP;
+ }
+ }
+
+ private BlockBalancedTreeWalker.TraversalState generateBalancedTree(int numRows, int leafSize, IntFunction<Integer> valueProvider) throws Exception
+ {
+ long treeOffset = writeBalancedTree(numRows, leafSize, valueProvider);
+
+ DataInput input = dataOutput.toDataInput();
+
+ input.skipBytes(treeOffset);
+ return new BlockBalancedTreeWalker.TraversalState(input);
+ }
+
+ private long writeBalancedTree(int numRows, int leafSize, IntFunction<Integer> valueProvider) throws Exception
+ {
+ final BlockBalancedTreeRamBuffer buffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+
+ byte[] scratch = new byte[4];
+ for (int rowID = 0; rowID < numRows; rowID++)
+ {
+ NumericUtils.intToSortableBytes(valueProvider.apply(rowID), scratch, 0);
+ buffer.add(rowID, scratch);
+ }
+
+ BlockBalancedTreeWriter writer = new BlockBalancedTreeWriter(4, leafSize);
+ ByteBuffersIndexOutput output = new ByteBuffersIndexOutput(dataOutput, "test", "test");
+ return writer.write(output, buffer.iterator(), (leafPostings, offset, count) -> {});
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java
new file mode 100644
index 0000000..86e8608
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/LeafOrderMapTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import org.apache.cassandra.index.sai.disk.ResettableByteBuffersIndexOutput;
+import org.apache.cassandra.index.sai.disk.io.SeekingRandomAccessInput;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersIndexInput;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.LongValues;
+import org.apache.lucene.util.packed.DirectReader;
+import org.apache.lucene.util.packed.DirectWriter;
+
+import static org.junit.Assert.assertEquals;
+
+public class LeafOrderMapTest extends SAIRandomizedTester
+{
+ @Test
+ public void test() throws Exception
+ {
+ int[] array = new int[1024];
+ for (int x=0; x < array.length; x++)
+ {
+ array[x] = x;
+ }
+ shuffle(array);
+
+ ResettableByteBuffersIndexOutput out = new ResettableByteBuffersIndexOutput("");
+
+ LeafOrderMap.write(array, array.length, array.length - 1, out);
+
+ IndexInput input = new ByteBuffersIndexInput(new ByteBuffersDataInput(Lists.newArrayList(ByteBuffer.wrap(out.toArrayCopy()))), "");
+
+ final byte bits = (byte) DirectWriter.unsignedBitsRequired(array.length - 1);
+
+ for (int index = 0; index < array.length; index++)
+ {
+ LongValues reader = DirectReader.getInstance(new SeekingRandomAccessInput(input), bits);
+
+ int value = Math.toIntExact(reader.get(index));
+
+ assertEquals(array[index], value);
+ }
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java
new file mode 100644
index 0000000..acaff3b
--- /dev/null
+++ b/test/unit/org/apache/cassandra/index/sai/disk/v1/bbtree/NumericIndexWriterTest.java
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.v1.bbtree;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import com.carrotsearch.hppc.LongArrayList;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.index.sai.IndexContext;
+import org.apache.cassandra.index.sai.QueryContext;
+import org.apache.cassandra.index.sai.SAITester;
+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.segment.SegmentMetadata;
+import org.apache.cassandra.index.sai.memory.MemtableTermsIterator;
+import org.apache.cassandra.index.sai.metrics.QueryEventListener;
+import org.apache.cassandra.index.sai.postings.PostingList;
+import org.apache.cassandra.index.sai.utils.SAIRandomizedTester;
+import org.apache.cassandra.index.sai.utils.TermsIterator;
+import org.apache.cassandra.index.sai.utils.TypeUtil;
+import org.apache.cassandra.io.util.FileHandle;
+import org.apache.cassandra.utils.AbstractGuavaIterator;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.NumericUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class NumericIndexWriterTest extends SAIRandomizedTester
+{
+ private IndexDescriptor indexDescriptor;
+ private IndexContext indexContext;
+
+ @Before
+ public void setup() throws Throwable
+ {
+ indexDescriptor = newIndexDescriptor();
+ indexContext = SAITester.createIndexContext(newIndex(), Int32Type.instance);
+ }
+
+ @Test
+ public void shouldFlushFromRamBuffer() throws Exception
+ {
+ final BlockBalancedTreeRamBuffer ramBuffer = new BlockBalancedTreeRamBuffer(Integer.BYTES);
+ final int numRows = 120;
+ int currentValue = numRows;
+ for (int i = 0; i < numRows; ++i)
+ {
+ byte[] scratch = new byte[Integer.BYTES];
+ NumericUtils.intToSortableBytes(currentValue--, scratch, 0);
+ ramBuffer.add(i, scratch);
+ }
+
+ int rowCount = ramBuffer.numRows();
+
+ SegmentMetadata.ComponentMetadataMap indexMetas;
+
+ NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ indexContext,
+ Integer.BYTES,
+ rowCount);
+ indexMetas = writer.writeCompleteSegment(ramBuffer.iterator());
+
+ final FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+ final FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+
+ try (BlockBalancedTreeReader reader = new BlockBalancedTreeReader(indexContext,
+ treeHandle,
+ indexMetas.get(IndexComponent.BALANCED_TREE).root,
+ treePostingsHandle,
+ indexMetas.get(IndexComponent.POSTING_LISTS).root))
+ {
+ final Counter visited = Counter.newCounter();
+ try (final PostingList ignored = reader.intersect(new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ // we should read point values in reverse order after sorting
+ assertEquals(1 + visited.get(), NumericUtils.sortableBytesToInt(packedValue, 0));
+ visited.addAndGet(1);
+ return true;
+ }
+
+ @Override
+ public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ return PointValues.Relation.CELL_CROSSES_QUERY;
+ }
+ }, mockEventListener(), mock(QueryContext.class)))
+ {
+ assertEquals(numRows, visited.get());
+ }
+ }
+ }
+
+ @Test
+ public void shouldFlushFromMemtable() throws Exception
+ {
+ final int maxSegmentRowId = 100;
+ final TermsIterator termEnum = buildTermEnum(0, maxSegmentRowId);
+
+ SegmentMetadata.ComponentMetadataMap indexMetas;
+ NumericIndexWriter writer = new NumericIndexWriter(indexDescriptor,
+ indexContext,
+ TypeUtil.fixedSizeOf(Int32Type.instance),
+ maxSegmentRowId);
+ indexMetas = writer.writeCompleteSegment(BlockBalancedTreeIterator.fromTermsIterator(termEnum, Int32Type.instance));
+
+ final FileHandle treeHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.BALANCED_TREE, indexContext);
+ final FileHandle treePostingsHandle = indexDescriptor.createPerIndexFileHandle(IndexComponent.POSTING_LISTS, indexContext);
+
+ try (BlockBalancedTreeReader reader = new BlockBalancedTreeReader(indexContext,
+ treeHandle,
+ indexMetas.get(IndexComponent.BALANCED_TREE).root,
+ treePostingsHandle,
+ indexMetas.get(IndexComponent.POSTING_LISTS).root
+ ))
+ {
+ final Counter visited = Counter.newCounter();
+ try (final PostingList ignored = reader.intersect(new BlockBalancedTreeReader.IntersectVisitor()
+ {
+ @Override
+ public boolean contains(byte[] packedValue)
+ {
+ final ByteComparable actualTerm = ByteComparable.fixedLength(packedValue);
+ final ByteComparable expectedTerm = ByteComparable.of(Math.toIntExact(visited.get()));
+ assertEquals("Point value mismatch after visiting " + visited.get() + " entries.", 0,
+ ByteComparable.compare(actualTerm, expectedTerm, ByteComparable.Version.OSS50));
+
+ visited.addAndGet(1);
+ return true;
+ }
+
+ @Override
+ public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue)
+ {
+ return PointValues.Relation.CELL_CROSSES_QUERY;
+ }
+ }, mockEventListener(), mock(QueryContext.class)))
+ {
+ assertEquals(maxSegmentRowId, visited.get());
+ }
+ }
+ }
+
+ private QueryEventListener.BalancedTreeEventListener mockEventListener()
+ {
+ QueryEventListener.BalancedTreeEventListener balancedTreeEventListener = mock(QueryEventListener.BalancedTreeEventListener.class);
+ when(balancedTreeEventListener.postingListEventListener()).thenReturn(mock(QueryEventListener.PostingListEventListener.class));
+ return balancedTreeEventListener;
+ }
+
+ private TermsIterator buildTermEnum(int startTermInclusive, int endTermExclusive)
+ {
+ final ByteBuffer minTerm = Int32Type.instance.decompose(startTermInclusive);
+ final ByteBuffer maxTerm = Int32Type.instance.decompose(endTermExclusive);
+
+ final AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>> iterator = new AbstractGuavaIterator<Pair<ByteComparable, LongArrayList>>()
+ {
+ private int currentTerm = startTermInclusive;
+ private int currentRowId = 0;
+
+ @Override
+ protected Pair<ByteComparable, LongArrayList> computeNext()
+ {
+ if (currentTerm >= endTermExclusive)
+ {
+ return endOfData();
+ }
+ final ByteBuffer term = Int32Type.instance.decompose(currentTerm++);
+ final LongArrayList postings = new LongArrayList();
+ postings.add(currentRowId++);
+ final ByteSource encoded = Int32Type.instance.asComparableBytes(term, ByteComparable.Version.OSS50);
+ return Pair.create(v -> encoded, postings);
+ }
+ };
+
+ return new MemtableTermsIterator(minTerm, maxTerm, iterator);
+ }
+}
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java b/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
index dfb22ac..e13cca5 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/CompactionTest.java
@@ -24,6 +24,7 @@
import java.util.Collections;
import com.google.common.collect.Lists;
+import org.junit.Assert;
import org.junit.Test;
import com.datastax.driver.core.exceptions.InvalidQueryException;
@@ -71,19 +72,19 @@
public void testAntiCompaction() throws Throwable
{
createTable(CREATE_TABLE_TEMPLATE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
verifyNoIndexFiles();
// create 100 rows in 1 sstable
int num = 100;
for (int i = 0; i < num; i++)
- execute( "INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute( "INSERT INTO %s (id1, v1) VALUES (?, 0)", Integer.toString(i));
flush();
// verify 1 sstable index
- assertNumRows(num, "SELECT * FROM %%s WHERE v1 = '0'");
- verifyIndexFiles(literalIndexContext, 1);
- verifySSTableIndexes(literalIndexContext.getIndexName(), 1);
+ assertNumRows(num, "SELECT * FROM %%s WHERE v1 >= 0");
+ verifyIndexFiles(numericIndexContext, null, 1, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1);
// split sstable into repaired and unrepaired
ColumnFamilyStore cfs = Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable());
@@ -108,24 +109,26 @@
}
// verify 2 sstable indexes
- assertNumRows(num, "SELECT * FROM %%s WHERE v1 = '0'");
- waitForAssert(() -> verifyIndexFiles(literalIndexContext, 2));
- verifySSTableIndexes(literalIndexContext.getIndexName(), 2);
+ assertNumRows(num, "SELECT * FROM %%s WHERE v1 >= 0");
+ waitForAssert(() -> verifyIndexFiles(numericIndexContext, null, 2, 0));
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 2);
// index components are included after anti-compaction
verifyIndexComponentsIncludedInSSTable();
}
@Test
- public void testConcurrentQueryWithCompaction() throws Throwable
+ public void testConcurrentQueryWithCompaction()
{
createTable(CREATE_TABLE_TEMPLATE);
String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+ waitForTableIndexesQueryable();
int num = 10;
for (int i = 0; i < num; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
flush();
}
@@ -134,7 +137,8 @@
{
try
{
- assertNumRows(num, "SELECT id1 FROM %s WHERE v1='0'");
+ assertNumRows(num, "SELECT id1 FROM %s WHERE v1>=0");
+ assertNumRows(num, "SELECT id1 FROM %s WHERE v2='0'");
}
catch (Throwable e)
{
@@ -146,6 +150,7 @@
compactionTest.start();
verifySSTableIndexes(v1IndexName, num);
+ verifySSTableIndexes(v2IndexName, num);
}
@Test
@@ -153,6 +158,7 @@
{
createTable(CREATE_TABLE_TEMPLATE);
String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
int sstables = 2;
int num = 10;
@@ -161,7 +167,7 @@
if (i == num / sstables)
flush();
- execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
}
flush();
@@ -186,11 +192,10 @@
}
catch (Throwable e)
{
- Throwable cause = e;
- while (cause.getCause() != null)
- cause = cause.getCause();
+ while (e.getCause() != null)
+ e = e.getCause();
- assertTrue(String.format("Expected %s, but got %s", errMessage, cause.getMessage()), cause.getMessage().contains(errMessage));
+ assertTrue(String.format("Expected %s, but got %s", errMessage, e.getMessage()), e.getMessage().contains(errMessage));
}
finally
{
@@ -200,8 +205,10 @@
assertNotEquals(0, earlyOpenCounter.get());
// verify indexes are working
- assertNumRows(num, "SELECT id1 FROM %%s WHERE v1='0'");
+ assertNumRows(num, "SELECT id1 FROM %%s WHERE v1=0");
+ assertNumRows(num, "SELECT id1 FROM %%s WHERE v2='0'");
verifySSTableIndexes(v1IndexName, sstables);
+ verifySSTableIndexes(v2IndexName, sstables);
}
@Test
@@ -214,7 +221,7 @@
int num = 100;
for (int i = 0; i < num; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
}
flush();
@@ -229,34 +236,36 @@
Injections.inject(compactionLatch);
TestWithConcurrentVerification compactionTask = new TestWithConcurrentVerification(
- () -> {
- try
- {
- upgradeSSTables();
- fail("Expected CompactionInterruptedException");
- }
- catch (Exception e)
- {
- assertTrue("Expected CompactionInterruptedException, but got " + e,
- Throwables.isCausedBy(e, CompactionInterruptedException.class::isInstance));
- }
- },
- () -> {
- try
- {
- waitForAssert(() -> assertEquals(1, compactionLatch.getCount()));
+ () -> {
+ try
+ {
+ upgradeSSTables();
+ fail("Expected CompactionInterruptedException");
+ }
+ catch (Exception e)
+ {
+ assertTrue("Expected CompactionInterruptedException, but got " + e,
+ Throwables.isCausedBy(e, CompactionInterruptedException.class::isInstance));
+ }
+ },
+ () -> {
+ try
+ {
+ waitForAssert(() -> Assert.assertEquals(1, compactionLatch.getCount()));
- // build indexes on SSTables that will be compacted soon
- createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ // build indexes on SSTables that will be compacted soon
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+ waitForTableIndexesQueryable();
- // continue in-progress compaction
- compactionLatch.countDown();
- }
- catch (Exception e)
- {
- throw new RuntimeException(e);
- }
- }, -1 // run verification task once
+ // continue in-progress compaction
+ compactionLatch.countDown();
+ }
+ catch (Exception e)
+ {
+ throw new RuntimeException(e);
+ }
+ }, -1 // run verification task once
);
compactionTask.start();
@@ -266,8 +275,10 @@
compactionLatch.disable();
}
- assertNumRows(num, "SELECT id1 FROM %%s WHERE v1='0'");
+ assertNumRows(num, "SELECT id1 FROM %%s WHERE v1>=0");
+ assertNumRows(num, "SELECT id1 FROM %%s WHERE v2='0'");
verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V1_COLUMN_IDENTIFIER), sstables);
+ verifySSTableIndexes(IndexMetadata.generateDefaultIndexName(currentTable(), V2_COLUMN_IDENTIFIER), sstables);
}
@Test
@@ -275,12 +286,14 @@
{
createTable(CREATE_TABLE_TEMPLATE);
String v1IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String v2IndexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+
// Load data into a single SSTable...
int num = 100;
for (int i = 0; i < num; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '0')", Integer.toString(i));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 0, '0')", Integer.toString(i));
}
flush();
@@ -288,32 +301,33 @@
assertNotEquals(0, getDiskUsage());
Injections.Barrier compactionLatch =
- Injections.newBarrier("pause_compaction_for_drop", 2, false)
- .add(InvokePointBuilder.newInvokePoint().onClass(SSTableIndexWriter.class).onMethod("addRow"))
- .build();
+ Injections.newBarrier("pause_compaction_for_drop", 2, false)
+ .add(InvokePointBuilder.newInvokePoint().onClass(SSTableIndexWriter.class).onMethod("addRow"))
+ .build();
try
{
// pause in-progress compaction
Injections.inject(compactionLatch);
TestWithConcurrentVerification compactionTask = new TestWithConcurrentVerification(
- this::upgradeSSTables,
- () -> {
- try
- {
- waitForAssert(() -> assertEquals(1, compactionLatch.getCount()));
+ this::upgradeSSTables,
+ () -> {
+ try
+ {
+ waitForAssert(() -> Assert.assertEquals(1, compactionLatch.getCount()));
- // drop all indexes
- dropIndex("DROP INDEX %s." + v1IndexName);
+ // drop all indexes
+ dropIndex("DROP INDEX %s." + v1IndexName);
+ dropIndex("DROP INDEX %s." + v2IndexName);
- // continue in-progress compaction
- compactionLatch.countDown();
- }
- catch (Throwable e)
- {
- throw new RuntimeException(e);
- }
- }, -1 // run verification task once
+ // continue in-progress compaction
+ compactionLatch.countDown();
+ }
+ catch (Throwable e)
+ {
+ throw new RuntimeException(e);
+ }
+ }, -1 // run verification task once
);
compactionTask.start();
@@ -330,8 +344,11 @@
// verify indexes are dropped
// verify indexes are dropped
- assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1='0'"))
- .isInstanceOf(InvalidQueryException.class)
- .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
+ assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v1>=0"))
+ .isInstanceOf(InvalidQueryException.class)
+ .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
+ assertThatThrownBy(() -> executeNet("SELECT id1 FROM %s WHERE v2='0'"))
+ .isInstanceOf(InvalidQueryException.class)
+ .hasMessage(StatementRestrictions.REQUIRES_ALLOW_FILTERING_MESSAGE);
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java b/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
index 152d472..1d052c1 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/DiskSpaceTest.java
@@ -33,7 +33,7 @@
int rows = 1000;
for (int j = 0; j < rows; j++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+ execute("INSERT INTO %s (id1, v1) VALUES (?, 1)", Integer.toString(j));
}
flush();
@@ -41,6 +41,7 @@
// create index, disk space should include index components
String indexName = createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ waitForTableIndexesQueryable();
long indexSize = indexDiskSpaceUse();
long sstableSizeWithIndex = totalDiskSpaceUsed();
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java b/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
index 191525e..b5322d3 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/DropTableTest.java
@@ -50,11 +50,13 @@
{
createTable(CREATE_TABLE_TEMPLATE);
createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2"));
+ waitForTableIndexesQueryable();
int rows = 100;
for (int j = 0; j < rows; j++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+ execute("INSERT INTO %s (id1, v1, v2) VALUES (?, 1 , '1')", Integer.toString(j));
}
flush();
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java b/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
index 893e304..16b3164 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/FailureTest.java
@@ -39,18 +39,18 @@
public void shouldMakeIndexNonQueryableOnSSTableContextFailureDuringFlush() throws Throwable
{
createTable(CREATE_TABLE_TEMPLATE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
- execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
+ execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
flush();
- assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+ assertEquals(1, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
- verifyIndexFiles(literalIndexContext, 1, 1, 1);
- verifySSTableIndexes(literalIndexContext.getIndexName(), 1, 1);
+ verifyIndexFiles(numericIndexContext, null, 1, 1, 0, 1, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1, 1);
- execute("INSERT INTO %s (id1, v1) VALUES ('3', '3')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('3', 3)");
Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_flush", SSTableContext.class, "<init>", RuntimeException.class);
Injections.inject(ssTableContextCreationFailure);
@@ -58,7 +58,7 @@
flush();
// Verify that, while the node is still operational, the index is not.
- Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+ Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1 > 1"))
.isInstanceOf(IndexNotAvailableException.class);
ssTableContextCreationFailure.disable();
@@ -66,28 +66,28 @@
// Now verify that a restart actually repairs the index...
simulateNodeRestart();
- verifyIndexFiles(literalIndexContext, 2);
- verifySSTableIndexes(literalIndexContext.getIndexName(), 2, 2);
+ verifyIndexFiles(numericIndexContext, null, 2, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 2, 2);
- assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+ assertEquals(2, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
}
@Test
public void shouldMakeIndexNonQueryableOnSSTableContextFailureDuringCompaction() throws Throwable
{
createTable(CREATE_TABLE_TEMPLATE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
flush();
- assertEquals(1, execute("SELECT id1 FROM %s WHERE v1='1'").size());
+ assertEquals(1, execute("SELECT id1 FROM %s WHERE v1 > 1").size());
- verifyIndexFiles(literalIndexContext, 2, 2, 2);
- verifySSTableIndexes(literalIndexContext.getIndexName(), 2, 2);
+ verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 2, 2);
Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_compaction", SSTableContext.class, "<init>", RuntimeException.class);
Injections.inject(ssTableContextCreationFailure);
@@ -95,7 +95,7 @@
compact();
// Verify that the index is not available.
- Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+ Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1 > 1"))
.isInstanceOf(IndexNotAvailableException.class);
}
@@ -104,22 +104,26 @@
{
createTable(CREATE_TABLE_TEMPLATE);
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
- execute("INSERT INTO %s (id1, v1) VALUES ('2', '2')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', 1)");
+ execute("INSERT INTO %s (id1, v1) VALUES ('2', 2)");
+
+ // We need to reference SSTableContext first or the failure injection fails
+ // because byteman can't find the class.
+ SSTableContext.class.getName();
Injection ssTableContextCreationFailure = newFailureOnEntry("context_failure_on_creation", SSTableContext.class, "<init>", RuntimeException.class);
Injections.inject(ssTableContextCreationFailure);
- String v1IndexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+ String v2IndexName = createIndexAsync(String.format(CREATE_INDEX_TEMPLATE, "v2"));
// Verify that the initial index build fails...
- verifyInitialIndexFailed(v1IndexName);
+ verifyInitialIndexFailed(v2IndexName);
verifyNoIndexFiles();
- verifySSTableIndexes(v1IndexName, 0);
+ verifySSTableIndexes(v2IndexName, 0);
// ...and then verify that, while the node is still operational, the index is not.
- Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v1='1'"))
+ Assertions.assertThatThrownBy(() -> execute("SELECT * FROM %s WHERE v2 = '1'"))
.isInstanceOf(IndexNotAvailableException.class);
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java b/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
index 7c067a5..79a126d 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/FlushingTest.java
@@ -23,38 +23,57 @@
import org.junit.Test;
import com.datastax.driver.core.ResultSet;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.index.sai.IndexContext;
import org.apache.cassandra.index.sai.SAITester;
+import org.apache.cassandra.index.sai.disk.v1.bbtree.NumericIndexWriter;
import static org.junit.Assert.assertEquals;
public class FlushingTest extends SAITester
{
@Test
+ public void testFlushingLargeStaleMemtableIndex() throws Throwable
+ {
+ createTable(CREATE_TABLE_TEMPLATE);
+ createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1"));
+
+ int overwrites = NumericIndexWriter.MAX_POINTS_IN_LEAF_NODE + 1;
+ for (int j = 0; j < overwrites; j++)
+ {
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', ?)", j);
+ }
+
+ flush();
+
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1>=0");
+ assertEquals(1, rows.all().size());
+ }
+
+ @Test
public void testFlushingOverwriteDelete() throws Throwable
{
createTable(CREATE_TABLE_TEMPLATE);
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
int sstables = 3;
for (int j = 0; j < sstables; j++)
{
- execute("INSERT INTO %s (id1, v1) VALUES (?, '1')", Integer.toString(j));
+ execute("INSERT INTO %s (id1, v1) VALUES (?, 1)", Integer.toString(j));
execute("DELETE FROM %s WHERE id1 = ?", Integer.toString(j));
flush();
}
- ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1 = '1'");
+ ResultSet rows = executeNet("SELECT id1 FROM %s WHERE v1 >= 0");
assertEquals(0, rows.all().size());
- verifyIndexFiles(literalIndexContext, sstables, 0, 3);
- verifySSTableIndexes(literalIndexContext.getIndexName(), sstables, 0);
+ verifyIndexFiles(numericIndexContext, null, sstables, 0, 0, sstables, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), sstables, 0);
compact();
- waitForAssert(() -> verifyIndexFiles(literalIndexContext, 1, 0, 1));
+ waitForAssert(() -> verifyIndexFiles(numericIndexContext, null, 1, 0, 0, 1, 0));
- rows = executeNet("SELECT id1 FROM %s WHERE v1 = '1'");
+ rows = executeNet("SELECT id1 FROM %s WHERE v1 >= 0");
assertEquals(0, rows.all().size());
- verifySSTableIndexes(literalIndexContext.getIndexName(), 1, 0);
+ verifySSTableIndexes(numericIndexContext.getIndexName(), 1, 0);
}
}
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 b8049b7..9e348dc 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/GroupComponentsTest.java
@@ -42,7 +42,7 @@
public class GroupComponentsTest extends SAITester
{
@Test
- public void testInvalidateWithoutObsolete() throws Throwable
+ public void testInvalidateWithoutObsolete()
{
createTable("CREATE TABLE %s (pk int primary key, value text)");
createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'");
@@ -67,7 +67,7 @@
}
@Test
- public void getLiveComponentsForEmptyIndex() throws Throwable
+ public void getLiveComponentsForEmptyIndex()
{
createTable("CREATE TABLE %s (pk int primary key, value text)");
createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'");
@@ -88,7 +88,7 @@
}
@Test
- public void getLiveComponentsForPopulatedIndex() throws Throwable
+ public void getLiveComponentsForPopulatedIndex()
{
createTable("CREATE TABLE %s (pk int primary key, value text)");
IndexContext indexContext = createIndexContext(createIndex("CREATE CUSTOM INDEX ON %s(value) USING 'StorageAttachedIndex'"), UTF8Type.instance);
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java b/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
index 1766b69..8ed9b6b 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/NodeRestartTest.java
@@ -23,6 +23,7 @@
import org.junit.Assert;
import org.junit.Test;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.index.sai.IndexContext;
import org.apache.cassandra.index.sai.SAITester;
@@ -92,7 +93,7 @@
waitForAssert(() -> Assert.assertEquals(1, initTaskLatch.getCount()));
// If we do not make the index queryable before it starts accepting queries, this will fail:
- assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
// Allow the init task to run, and then wait for it to finish...
initTaskLatch.countDown();
@@ -108,19 +109,23 @@
createTable(CREATE_TABLE_TEMPLATE);
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0');");
flush();
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 1);
- assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v2")), UTF8Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext,literalIndexContext, 1, 1);
+ assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
+ assertNumRows(1, "SELECT * FROM %%s WHERE v2 = '0'");
assertValidationCount(0, 0);
simulateNodeRestart();
- verifyIndexFiles(literalIndexContext, 1);
+ verifyIndexFiles(numericIndexContext, literalIndexContext, 1, 1);
- assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
+ assertNumRows(1, "SELECT * FROM %%s WHERE v2 = '0'");
waitForTableIndexesQueryable();
@@ -158,7 +163,7 @@
// This will fail if the pre-join task doesn't skip validation (after the init task has already run):
assertValidationCount(0, 0);
- assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
}
void createSingleRowIndex() throws Throwable
@@ -166,12 +171,13 @@
createTable(CREATE_TABLE_TEMPLATE);
verifyNoIndexFiles();
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
flush();
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 1);
- assertNumRows(1, "SELECT * FROM %%s WHERE v1 = '0'");
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext, null, 1, 0);
+ assertNumRows(1, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(0, 0);
}
}
diff --git a/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java b/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
index 769435d..cd20955 100644
--- a/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/functional/SnapshotTest.java
@@ -21,7 +21,7 @@
import org.junit.Before;
import org.junit.Test;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.Int32Type;
import org.apache.cassandra.index.sai.IndexContext;
import org.apache.cassandra.index.sai.SAITester;
import org.apache.cassandra.inject.Injections;
@@ -50,60 +50,60 @@
verifyNoIndexFiles();
// Insert some initial data and create the index over it
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
+ execute("INSERT INTO %s (id1, v1) VALUES ('0', 0);");
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ waitForTableIndexesQueryable();
flush();
- verifyIndexFiles(literalIndexContext, 1, 1, 1);
+ verifyIndexFiles(numericIndexContext, null, 1, 1, 0, 1, 0);
// Note: This test will fail here if it is run on its own because the per-index validation
// is run if the node is starting up but validatation isn't done once the node is started
assertValidationCount(0, 0);
resetValidationCount();
// Add some data into a second sstable
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', 0);");
flush();
- verifyIndexFiles(literalIndexContext, 2, 2, 2);
+ verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
assertValidationCount(0, 0);
// Take a snapshot recording the index files last modified date
String snapshot = "s";
- int numSnapshottedSSTables = snapshot(snapshot);
- assertEquals(1, numSnapshottedSSTables);
+ assertEquals(1, snapshot(snapshot));
long snapshotLastModified = indexFilesLastModified();
- // File.lastModified result can be truncated one second resolution, which can be less than the index build
+ // File.lastModified result can be truncated one second resolution, which can be lesser than the index build
// time, so we sleep for that time to guarantee that the modification date any of overridden index file will be
// different to that of the original file
Thread.sleep(1000);
// Add some data into a third sstable, out of the scope of our snapshot
- execute("INSERT INTO %s (id1, v1) VALUES ('2', '0');");
+ execute("INSERT INTO %s (id1, v1) VALUES ('2', 0);");
flush();
- verifyIndexFiles(literalIndexContext, 3, 3, 3);
- assertNumRows(3, "SELECT * FROM %%s WHERE v1 = '0'");
+ verifyIndexFiles(numericIndexContext, null, 3, 3, 0, 3, 0);
+ assertNumRows(3, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(0, 0);
// Truncate the table
truncate(false);
- waitForAssert(this::verifyNoIndexFiles);
- assertNumRows(0, "SELECT * FROM %%s WHERE v1 = '0'");
+ waitForAssert(() -> verifyNoIndexFiles());
+ assertNumRows(0, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(0, 0);
// Restore the snapshot, only the two first sstables should be restored
restoreSnapshot(snapshot);
- verifyIndexFiles(literalIndexContext, 2, 2, 2);
+ verifyIndexFiles(numericIndexContext, null, 2, 2, 0, 2, 0);
assertEquals(snapshotLastModified, indexFilesLastModified());
- assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(2, 2); // newly loaded
// index components are included after restore
verifyIndexComponentsIncludedInSSTable();
// Rebuild the index to verify that the index files are overridden
- rebuildIndexes(literalIndexContext.getIndexName());
- verifyIndexFiles(literalIndexContext, 2);
+ rebuildIndexes(numericIndexContext.getIndexName());
+ verifyIndexFiles(numericIndexContext, null, 2, 0);
assertNotEquals(snapshotLastModified, indexFilesLastModified());
- assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(2, 2); // compaction should not validate
// index components are included after rebuild
@@ -117,19 +117,20 @@
verifyNoIndexFiles();
// Insert some initial data
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0');");
+ execute("INSERT INTO %s (id1, v1) VALUES ('0', 0);");
flush();
// Add some data into a second sstable
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '0');");
+ execute("INSERT INTO %s (id1, v1) VALUES ('1', 0);");
flush();
// index components are not included
verifyIndexComponentsNotIncludedInSSTable();
// create index
- IndexContext literalIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), UTF8Type.instance);
- verifyIndexFiles(literalIndexContext, 2);
+ IndexContext numericIndexContext = createIndexContext(createIndex(String.format(CREATE_INDEX_TEMPLATE, "v1")), Int32Type.instance);
+ waitForTableIndexesQueryable();
+ verifyIndexFiles(numericIndexContext, null, 2, 0);
assertValidationCount(0, 0);
// index components are included after initial build
@@ -137,8 +138,7 @@
// Take a snapshot recording the index files last modified date
String snapshot = "s";
- int numSnapshottedSSTables = snapshot(snapshot);
- assertEquals(1, numSnapshottedSSTables);
+ assertEquals(1, snapshot(snapshot));
long snapshotLastModified = indexFilesLastModified();
// File.lastModified result can be truncated one second resolution, which can be lesser than the index build
@@ -148,15 +148,15 @@
// Truncate the table
truncate(false);
- waitForAssert(this::verifyNoIndexFiles);
- assertNumRows(0, "SELECT * FROM %%s WHERE v1 = '0'");
+ waitForAssert(() -> verifyNoIndexFiles());
+ assertNumRows(0, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(0, 0);
// Restore the snapshot
restoreSnapshot(snapshot);
- verifyIndexFiles(literalIndexContext, 2);
+ verifyIndexFiles(numericIndexContext, null, 2, 0);
assertEquals(snapshotLastModified, indexFilesLastModified());
- assertNumRows(2, "SELECT * FROM %%s WHERE v1 = '0'");
+ assertNumRows(2, "SELECT * FROM %%s WHERE v1 >= 0");
assertValidationCount(2, 2); // newly loaded
// index components are included after restore snapshot
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 7b28aaf..57ad70e 100644
--- a/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/metrics/IndexGroupMetricsTest.java
@@ -54,7 +54,7 @@
int sstables = 10;
for (int i = 0; i < sstables; i++)
{
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO %s (id1, v1) VALUES ('0', 0)");
flush();
}
diff --git a/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java b/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
index 6bc6c99..378bbd4 100644
--- a/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
+++ b/test/unit/org/apache/cassandra/index/sai/metrics/SegmentFlushingFailureTester.java
@@ -97,14 +97,14 @@
assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
- assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '1')");
flush();
- ResultSet rows = executeNet("SELECT * FROM %s WHERE v1 = '0'");
+ ResultSet rows = executeNet("SELECT * FROM %s WHERE v1 = 0");
assertEquals(1, rows.all().size());
compact();
@@ -116,9 +116,9 @@
assertTrue(memoryTrackingCounter.get() > 0);
assertEquals("Global memory tracker should have reverted to zero.", 0L, getSegmentBufferUsedBytes());
- assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
- rows = executeNet("SELECT * FROM %s WHERE v1 = '0'");
+ rows = executeNet("SELECT * FROM %s WHERE v1 = 0");
assertEquals(1, rows.all().size());
}
@@ -147,17 +147,17 @@
assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
- assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
- execute("INSERT INTO %s (id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('0', 0, '0')");
flush();
- execute("INSERT INTO %s (id1, v1) VALUES ('1', '1')");
+ execute("INSERT INTO %s (id1, v1, v2) VALUES ('1', 1, '1')");
flush();
// Verify that we abort exactly once and zero the memory tracker:
verifyCompactionIndexBuilds(1, failure, currentTable());
- String select = String.format("SELECT * FROM %%s WHERE %s = %s", column, "'0'");
+ String select = String.format("SELECT * FROM %%s WHERE %s = %s", column, column.equals("v1") ? "0" : "'0'");
assertThatThrownBy(() -> executeNet(select)).isInstanceOf(ReadFailureException.class);
}
@@ -172,26 +172,26 @@
assertEquals(expectedBytesLimit(), getSegmentBufferSpaceLimit());
assertEquals("Segment buffer memory tracker should start at zero!", 0L, getSegmentBufferUsedBytes());
- assertEquals("There should be no segment builders in progress.", 0, getColumnIndexBuildsInProgress());
+ assertEquals("There should be no segment builders in progress.", 0L, getColumnIndexBuildsInProgress());
- execute("INSERT INTO " + KEYSPACE + '.' + table1 + "(id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO " + KEYSPACE + "." + table1 + "(id1, v1, v2) VALUES ('0', 0, '0')");
flush(KEYSPACE, table1);
- execute("INSERT INTO " + KEYSPACE + '.' + table1 + "(id1, v1) VALUES ('1', '1')");
+ execute("INSERT INTO " + KEYSPACE + "." + table1 + "(id1, v1, v2) VALUES ('1', 1, '1')");
flush(KEYSPACE, table1);
- execute("INSERT INTO " + KEYSPACE + '.' + table2 + "(id1, v1) VALUES ('0', '0')");
+ execute("INSERT INTO " + KEYSPACE + "." + table2 + "(id1, v1, v2) VALUES ('0', 0, '0')");
flush(KEYSPACE, table2);
- execute("INSERT INTO " + KEYSPACE + '.' + table2 + "(id1, v1) VALUES ('1', '1')");
+ execute("INSERT INTO " + KEYSPACE + "." + table2 + "(id1, v1, v2) VALUES ('1', 1, '1')");
flush(KEYSPACE, table2);
// Start compaction against both tables/indexes and verify that they are aborted safely:
verifyCompactionIndexBuilds(2, segmentFlushFailure, table1, table2);
- assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s.%s WHERE v1 = '0'", KEYSPACE, table1)))
- .isInstanceOf(ReadFailureException.class);
+ assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s WHERE v1 = 0", KEYSPACE + "." + table1)))
+ .isInstanceOf(ReadFailureException.class);
- assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s.%s WHERE v1 = '0'", KEYSPACE, table2)))
- .isInstanceOf(ReadFailureException.class);
+ assertThatThrownBy(() -> executeNet(String.format("SELECT * FROM %s WHERE v1 = 0", KEYSPACE + "." + table2)))
+ .isInstanceOf(ReadFailureException.class);
}
private void verifyCompactionIndexBuilds(int aborts, Injection failure, String... tables) throws Throwable
diff --git a/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java b/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
index f8db85f..636f603 100644
--- a/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
+++ b/test/unit/org/apache/cassandra/index/sai/utils/SAIRandomizedTester.java
@@ -121,6 +121,11 @@
return between(min, max - 1);
}
+ public static int between(int min, int max)
+ {
+ return getRandom().nextIntBetween(min, max - 1);
+ }
+
public static long between(long min, long max)
{
return randomLongBetween(min, max);
diff --git a/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java b/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
index 9f91c27..69922ee 100644
--- a/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
+++ b/test/unit/org/apache/cassandra/index/sai/utils/TypeUtilTest.java
@@ -63,8 +63,8 @@
boolean isLiteral = cql3Type == CQL3Type.Native.ASCII || cql3Type == CQL3Type.Native.TEXT || cql3Type == CQL3Type.Native.VARCHAR || cql3Type == CQL3Type.Native.BOOLEAN;
assertEquals(isLiteral, TypeUtil.isLiteral(type));
assertEquals(TypeUtil.isLiteral(type), TypeUtil.isLiteral(reversedType));
- assertEquals(isUTF8OrAscii, TypeUtil.isUTF8OrAscii(type));
- assertEquals(TypeUtil.isUTF8OrAscii(type), TypeUtil.isUTF8OrAscii(reversedType));
+ assertEquals(isUTF8OrAscii, TypeUtil.isString(type));
+ assertEquals(TypeUtil.isString(type), TypeUtil.isString(reversedType));
}
}