Skip to content

Commit

Permalink
CNDB-12041: Add missing check of guardrail sai_sstable_indexes_per_query
Browse files Browse the repository at this point in the history
  • Loading branch information
adelapena committed Jan 15, 2025
1 parent 5700159 commit 9ed6534
Show file tree
Hide file tree
Showing 5 changed files with 58 additions and 26 deletions.
21 changes: 21 additions & 0 deletions src/java/org/apache/cassandra/index/sai/plan/FilterTree.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,18 @@

import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.ListIterator;
import java.util.Set;

import com.google.common.collect.ListMultimap;

import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.rows.Row;
import org.apache.cassandra.db.rows.Unfiltered;
import org.apache.cassandra.index.sai.SSTableIndex;
import org.apache.cassandra.index.sai.utils.TypeUtil;
import org.apache.cassandra.schema.ColumnMetadata;
import org.apache.cassandra.schema.ColumnMetadata.Kind;
Expand Down Expand Up @@ -130,4 +133,22 @@ private boolean localSatisfiedBy(DecoratedKey key, Unfiltered currentCluster, Ro
private boolean shouldReturnNow(boolean result) {
return (op == OperationType.AND && !result) || (op == OperationType.OR && result);
}

/**
* @return the number of unique SSTable indexes that are referenced by the expressions in this filter tree.
*/
public int numSSTableIndexes()
{
Set<SSTableIndex> referencedIndexes = new HashSet<>();
sstableIndexes(referencedIndexes);
return referencedIndexes.size();
}

private void sstableIndexes(Set<SSTableIndex> indexes)
{
for (Expression expression : expressions.values())
indexes.addAll(expression.context.getView().getIndexes());
for (FilterTree child : children)
child.sstableIndexes(indexes);
}
}
22 changes: 0 additions & 22 deletions src/java/org/apache/cassandra/index/sai/plan/QueryController.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.DataRange;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.MessageParams;
import org.apache.cassandra.db.MultiRangeReadCommand;
import org.apache.cassandra.db.PartitionPosition;
import org.apache.cassandra.db.PartitionRangeReadCommand;
Expand All @@ -55,7 +54,6 @@
import org.apache.cassandra.db.filter.ClusteringIndexSliceFilter;
import org.apache.cassandra.db.filter.DataLimits;
import org.apache.cassandra.db.filter.RowFilter;
import org.apache.cassandra.db.guardrails.Guardrails;
import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.db.marshal.AbstractType;
import org.apache.cassandra.db.marshal.CollectionType;
Expand Down Expand Up @@ -87,7 +85,6 @@
import org.apache.cassandra.index.sai.view.View;
import org.apache.cassandra.io.sstable.format.SSTableReader;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.net.ParamType;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.CloseableIterator;
Expand Down Expand Up @@ -822,25 +819,6 @@ private void closeQueryViews()
}
}

void maybeTriggerReferencedIndexesGuardrail(int numReferencedIndexes)
{
if (Guardrails.saiSSTableIndexesPerQuery.failsOn(numReferencedIndexes, null))
{
String msg = String.format("Query %s attempted to read from too many indexes (%s) but max allowed is %s; " +
"query aborted (see sai_sstable_indexes_per_query_fail_threshold)",
command.toCQLString(),
numReferencedIndexes,
Guardrails.CONFIG_PROVIDER.getOrCreate(null).getSaiSSTableIndexesPerQueryFailThreshold());
Tracing.trace(msg);
MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_FAIL, numReferencedIndexes);
throw new QueryReferencingTooManyIndexesException(msg);
}
else if (Guardrails.saiSSTableIndexesPerQuery.warnsOn(numReferencedIndexes, null))
{
MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_WARN, numReferencedIndexes);
}
}

/**
* Returns the {@link DataRange} list covered by the specified {@link ReadCommand}.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,11 @@
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.DataRange;
import org.apache.cassandra.db.DecoratedKey;
import org.apache.cassandra.db.MessageParams;
import org.apache.cassandra.db.PartitionPosition;
import org.apache.cassandra.db.ReadCommand;
import org.apache.cassandra.db.ReadExecutionController;
import org.apache.cassandra.db.guardrails.Guardrails;
import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator;
import org.apache.cassandra.db.rows.Row;
Expand All @@ -63,7 +65,9 @@
import org.apache.cassandra.index.sai.utils.RangeUtil;
import org.apache.cassandra.index.sai.utils.PrimaryKeyWithSortKey;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.net.ParamType;
import org.apache.cassandra.schema.TableMetadata;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.AbstractIterator;
import org.apache.cassandra.utils.Clock;
import org.apache.cassandra.utils.CloseableIterator;
Expand Down Expand Up @@ -109,6 +113,8 @@ public UnfilteredPartitionIterator search(ReadExecutionController executionContr
try
{
FilterTree filterTree = analyzeFilter();
maybeTriggerReferencedIndexesGuardrail(filterTree);

Plan plan = controller.buildPlan();
Iterator<? extends PrimaryKey> keysIterator = controller.buildIterator(plan);

Expand Down Expand Up @@ -154,6 +160,30 @@ public UnfilteredPartitionIterator search(ReadExecutionController executionContr
}
}

private void maybeTriggerReferencedIndexesGuardrail(FilterTree filterTree)
{
if (!Guardrails.saiSSTableIndexesPerQuery.enabled())
return;

int numReferencedIndexes = filterTree.numSSTableIndexes();

if (Guardrails.saiSSTableIndexesPerQuery.failsOn(numReferencedIndexes, null))
{
String msg = String.format("Query %s attempted to read from too many indexes (%s) but max allowed is %s; " +
"query aborted (see sai_sstable_indexes_per_query_fail_threshold)",
command.toCQLString(),
numReferencedIndexes,
Guardrails.CONFIG_PROVIDER.getOrCreate(null).getSaiSSTableIndexesPerQueryFailThreshold());
Tracing.trace(msg);
MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_FAIL, numReferencedIndexes);
throw new QueryReferencingTooManyIndexesException(msg);
}
else if (Guardrails.saiSSTableIndexesPerQuery.warnsOn(numReferencedIndexes, null))
{
MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_WARN, numReferencedIndexes);
}
}

/**
* Converts expressions into filter tree (which is currently just a single AND).
* </p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,7 +194,10 @@ public int hashCode()
@Override
public String toString()
{
return "(tombstones=" + tombstones + ", localReadSize=" + localReadSize + ", rowIndexTooLarge=" + rowIndexReadSize + ')';
return "(tombstones=" + tombstones +
", localReadSize=" + localReadSize +
", rowIndexTooLarge=" + rowIndexReadSize +
", indexReadSSTablesCount=" + indexReadSSTablesCount + ')';
}

public static final class Warnings
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -144,11 +144,11 @@ public void testSAIWarnThreshold()

assertWarnAborts(0, 0);

// create 3 more SSTables on each node, this will trigger warn threshold (3 > 2 but < 5)
// create 3 more SSTables on each node, this will trigger warn threshold (4 > 2 but < 5)
valueToQuery = createSSTables(3);
String valueToQueryString = LongType.instance.toCQLString(LongType.instance.decompose(valueToQuery), true);
String expectedMessage = tooManyIndexesReadWarnMessage(cluster.size(),
3,
4,
String.format("SELECT * FROM %s.%s WHERE v1 = %s ALLOW FILTERING",
KEYSPACE, tableName, valueToQueryString));
assertThat(getOnlyElement(executeSelect(valueToQuery, false))).contains(expectedMessage);
Expand All @@ -168,7 +168,7 @@ public void testSAIWarnThreshold()

// notice we expect warnings from 2 nodes
expectedMessage = tooManyIndexesReadWarnMessage(cluster.size() - 1,
3,
4,
String.format("SELECT * FROM %s.%s WHERE v1 = %s ALLOW FILTERING",
KEYSPACE, tableName, valueToQueryString));

Expand Down

0 comments on commit 9ed6534

Please sign in to comment.