[ https://issues.apache.org/jira/browse/CASSANDRA-13794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16166838#comment-16166838 ]
Benedict edited comment on CASSANDRA-13794 at 9/14/17 7:13 PM: --------------------------------------------------------------- This patch is great (excepting a couple of extraneous edits). Love the comments. +1 I would suggest filing two follow-up tickets to address some short comings with this code path, but they're edge-case, and not straight-forward enough, to not block this merge. # For some extreme users, 16 rows could be a huge amount of data. There should probably be some modulation of this lower bound based on known data sizes in the table, or the like. # Conversely, in an overloaded cluster on which users are commonly performing fairly large-limit reads (say, 1k+ moderate sized rows) of larger partitions, we could find ourselves doubling the amount of work the cluster needs to do; during overload we can expect dropped writes, and a single missing row in any read would trigger a same-sized read. This could iteratively compound the overload. The best solutions to this problem are probably non-trivial, though a simplish approach might be to use exponential growth, bounded on both sides by a minimum and maximum (perhaps similarly determined from the known data size distribution) - with the query limit being used as the first value if it is small enough. I _would_ say that (2) is no worse than the status-quo, given the per-request overheads are probably greater than the per-datum overheads in a typical cluster, but CASSANDRA-12872 suggests we haven't been incurring the full overheads of SRP, so we cannot claim that. I still think it is reasonable to address this in a follow-up ticket, however. was (Author: benedict): This patch is great (excepting a couple of extraneous edits). Love the comments. +1 I would suggest filing two follow-up tickets to address some short comings with this code path, but they're edge-case, and not straight-forward enough, to not block this merge. # For some extreme users, 16 rows could be a huge amount of data. There should probably be some modulation of this lower bound based on known data sizes in the table, or the like. # Conversely, in an overloaded cluster on which users are commonly performing fairly large-limit reads (say, 1k+ moderate sized rows) of larger partitions, we could find ourselves doubling the amount of work the cluster needs to do; during overload we can expect dropped writes, and a single missing row in any read would trigger a same-sized read. This could iteratively compound the overload. The best solutions to this problem are probably non-trivial, though a simplish approach might be to use exponential growth, bounded on both sides by a minimum and maximum (perhaps similarly determined from the known data size distribution) - with the query limit being used as the first value if it is small enough. I _would_ say that (2) is now worse than the status-quo, given the per-request overheads are probably greater than the per-datum overheads in a typical cluster, but CASSANDRA-12872 suggests we haven't been incurring the full overheads of SRP, so we cannot claim that. I still think it is reasonable to address this in a follow-up ticket, however. > Fix short read protection logic for querying more rows > ------------------------------------------------------ > > Key: CASSANDRA-13794 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13794 > Project: Cassandra > Issue Type: Bug > Components: Coordination > Reporter: Benedict > Assignee: Aleksey Yeschenko > Labels: Correctness > Fix For: 3.0.x, 3.11.x > > > Discovered by [~benedict] while reviewing CASSANDRA-13747: > {quote} > While reviewing I got a little suspicious of the modified line > {{DataResolver}} :479, as it seemed that n and x were the wrong way around... > and, reading the comment of intent directly above, and reproducing the > calculation, they are indeed. > This is probably a significant enough bug that it warrants its own ticket for > record keeping, though I'm fairly agnostic on that decision. > I'm a little concerned about our current short read behaviour, as right now > it seems we should be requesting exactly one row, for any size of under-read, > which could mean extremely poor performance in case of large under-reads. > I would suggest that the outer unconditional {{Math.max}} is a bad idea, has > been (poorly) insulating us from this error, and that we should first be > asserting that the calculation yields a value >= 0 before setting to 1. > {quote} -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org