maedhroz commented on code in PR #3916: URL: https://github.com/apache/cassandra/pull/3916#discussion_r1964524913
########## src/java/org/apache/cassandra/service/reads/ReplicaFilteringProtection.java: ########## @@ -538,14 +538,19 @@ private UnfilteredPartitionIterator fetchFromSource() Tracing.trace("Requesting {} rows in partition {} from {} for replica filtering protection", clusterings.size(), key, source); - // build the read command taking into account that we could be requesting only in the static row - DataLimits limits = clusterings.isEmpty() ? DataLimits.cqlLimits(1) : DataLimits.NONE; - ClusteringIndexFilter filter = unresolvedStatic ? command.clusteringIndexFilter(key) : new ClusteringIndexNamesFilter(clusterings, command.isReversed()); + // If there is an unresolved static column, we must fetch the entire partition, as static column predicates + // may produce row matches across the entire partition. If there are only non-static rows to complete, we + // query the partition specifically for the corresponding cluterings by name. In either case, we do not + // provide a limit. (In the unresolved static case, we have no way of knowing how many stale rows we might + // read on a silent replica before finding a live one.) + ClusteringIndexFilter filter = unresolvedStatic ? command.clusteringIndexFilter(key) + : new ClusteringIndexNamesFilter(clusterings, command.isReversed()); Review Comment: This is the crux of the patch. I went back and forth a bit, but I don't know if we can avoid fetching the rest of the partition (in the current data range). In practice, we're not going to be using ridiculously small fetch sizes, so I don't know how much of a problem this will actually be, but the viral nature of static columns is what it is. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org For additional commands, e-mail: pr-h...@cassandra.apache.org