Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-3852

use LIFO queueing policy when queue size exceeds thresholds

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Open
    • Normal
    • Resolution: Unresolved
    • 5.x
    • Legacy/Core

    Description

      A strict FIFO policy for queueing (between stages) is detrimental to latency and forward progress. Whenever a node is saturated beyond incoming request rate, all requests become slow. If it is consistently saturated, you start effectively timing out on all requests.

      A much better strategy from the point of view of latency is to serve a subset requests quickly, and letting some time out, rather than letting all either time out or be slow.

      Care must be taken such that:

      • We still guarantee that requests are processed reasonably timely (we couldn't go strict LIFO for example as that would result in requests getting stuck potentially forever on a loaded node).
      • Maybe, depending on the previous point's solution, ensure that some requests bypass the policy and get prioritized (e.g., schema migrations, or anything "internal" to a node).

      A possible implementation is to go LIFO whenever there are requests in the queue that are older than N milliseconds (or a certain queue size, etc).

      Benefits:

      • All cases where the client is directly, or is indirectly affecting through other layers, a system which has limited concurrency (e.g., thread pool size of X to serve some incoming request rate), it is much better for a few requests to time out while most are serviced quickly, than for all requests to become slow, as it doesn't explode concurrency. Think any random non-super-advanced php app, ruby web app, java servlet based app, etc. Essentially, it optimizes very heavily for improved average latencies.
      • Systems with strict p95/p99/p999 requirements on latencies should greatly benefit from such a policy. For example, suppose you have a system at 85% of capacity, and it takes a write spike (or has a hiccup like GC pause, blocking on a commit log write, etc). Suppose the hiccup racks up 500 ms worth of requests. At 15% margin at steady state, that takes 500ms * 100/15 = 3.2 seconds to recover. Instead of all requests for an entire 3.2 second window being slow, we'd serve requests quickly for 2.7 of those seconds, with the incoming requests during that 500 ms interval being the ones primarily affected. The flip side though is that once you're at the point where more than N percent of requests end up having to wait for others to take LIFO priority, the p(100-N) latencies will actually be worse than without this change (but at this point you have to consider what the root reason for those pXX requirements are).
      • In the case of complete saturation, it allows forward progress. Suppose you're taking 25% more traffic than you are able to handle. Instead of getting backed up and ending up essentially timing out every single request, you will succeed in processing up to 75% of them (I say "up to" because it depends; for example on a QUORUM request you need at least two of the requests from the co-ordinator to succeed so the percentage is brought down) and allowing clients to make forward progress and get work done, rather than being stuck.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              scode Peter Schuller
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated: