|
42 | 42 | import org.apache.cassandra.db.ColumnFamilyStore;
|
43 | 43 | import org.apache.cassandra.db.DataRange;
|
44 | 44 | import org.apache.cassandra.db.DecoratedKey;
|
| 45 | +import org.apache.cassandra.db.MessageParams; |
45 | 46 | import org.apache.cassandra.db.PartitionPosition;
|
46 | 47 | import org.apache.cassandra.db.ReadCommand;
|
47 | 48 | import org.apache.cassandra.db.ReadExecutionController;
|
| 49 | +import org.apache.cassandra.db.guardrails.Guardrails; |
48 | 50 | import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
|
49 | 51 | import org.apache.cassandra.db.rows.AbstractUnfilteredRowIterator;
|
50 | 52 | import org.apache.cassandra.db.rows.Row;
|
|
63 | 65 | import org.apache.cassandra.index.sai.utils.RangeUtil;
|
64 | 66 | import org.apache.cassandra.index.sai.utils.PrimaryKeyWithSortKey;
|
65 | 67 | import org.apache.cassandra.io.util.FileUtils;
|
| 68 | +import org.apache.cassandra.net.ParamType; |
66 | 69 | import org.apache.cassandra.schema.TableMetadata;
|
| 70 | +import org.apache.cassandra.tracing.Tracing; |
67 | 71 | import org.apache.cassandra.utils.AbstractIterator;
|
68 | 72 | import org.apache.cassandra.utils.Clock;
|
69 | 73 | import org.apache.cassandra.utils.CloseableIterator;
|
@@ -109,6 +113,8 @@ public UnfilteredPartitionIterator search(ReadExecutionController executionContr
|
109 | 113 | try
|
110 | 114 | {
|
111 | 115 | FilterTree filterTree = analyzeFilter();
|
| 116 | + maybeTriggerReferencedIndexesGuardrail(filterTree); |
| 117 | + |
112 | 118 | Plan plan = controller.buildPlan();
|
113 | 119 | Iterator<? extends PrimaryKey> keysIterator = controller.buildIterator(plan);
|
114 | 120 |
|
@@ -154,6 +160,30 @@ public UnfilteredPartitionIterator search(ReadExecutionController executionContr
|
154 | 160 | }
|
155 | 161 | }
|
156 | 162 |
|
| 163 | + private void maybeTriggerReferencedIndexesGuardrail(FilterTree filterTree) |
| 164 | + { |
| 165 | + if (!Guardrails.saiSSTableIndexesPerQuery.enabled()) |
| 166 | + return; |
| 167 | + |
| 168 | + int numReferencedIndexes = filterTree.numSSTableIndexes(); |
| 169 | + |
| 170 | + if (Guardrails.saiSSTableIndexesPerQuery.failsOn(numReferencedIndexes, null)) |
| 171 | + { |
| 172 | + String msg = String.format("Query %s attempted to read from too many indexes (%s) but max allowed is %s; " + |
| 173 | + "query aborted (see sai_sstable_indexes_per_query_fail_threshold)", |
| 174 | + command.toCQLString(), |
| 175 | + numReferencedIndexes, |
| 176 | + Guardrails.CONFIG_PROVIDER.getOrCreate(null).getSaiSSTableIndexesPerQueryFailThreshold()); |
| 177 | + Tracing.trace(msg); |
| 178 | + MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_FAIL, numReferencedIndexes); |
| 179 | + throw new QueryReferencingTooManyIndexesException(msg); |
| 180 | + } |
| 181 | + else if (Guardrails.saiSSTableIndexesPerQuery.warnsOn(numReferencedIndexes, null)) |
| 182 | + { |
| 183 | + MessageParams.add(ParamType.TOO_MANY_REFERENCED_INDEXES_WARN, numReferencedIndexes); |
| 184 | + } |
| 185 | + } |
| 186 | + |
157 | 187 | /**
|
158 | 188 | * Converts expressions into filter tree (which is currently just a single AND).
|
159 | 189 | * </p>
|
|
0 commit comments