github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/docs/RFCS/20170719_distsql_buffering_router.md (about)

     1  - Feature Name: distsql buffering hash router
     2  - Status: completed
     3  - Start Date: 2017-07-19
     4  - Authors: Radu
     5  - RFC PR: [#17105](https://github.com/cockroachdb/cockroach/pull/17105)
     6  - Cockroach Issue: [#17097](https://github.com/cockroachdb/cockroach/issues/17097)
     7  
     8  # Summary
     9  
    10  This RFC discusses the implementation of a "by-hash" output router in distsql
    11  which doesn't stop sending results once a consumer is blocked.
    12  
    13  # Motivation
    14  
    15  Issue [#17097](https://github.com/cockroachdb/cockroach/issues/17097) describes
    16  scenarios in which a distsql computation can deadlock. The crux of the issue is
    17  that the streams between the processors have limited buffers, and when sending
    18  on one of these streams blocks, it can block a producer with multiple consumers;
    19  in some cases, sending rows to one of the non-blocked consumers is required for
    20  progress, so the current implementation deadlocks. See the issue for some
    21  examples.
    22  
    23  We can fix this by adding buffering on the input side, wherever a consumer reads
    24  from multiple producers (synchronizers, joiners). However, it is difficult to
    25  determine when we need to buffer (and we don't want to buffer unnecessarily);
    26  and, there are multiple distsql components that would be affected.
    27  
    28  The alternative is to buffer on the output side; currently, the only component
    29  which has multiple consumers is the hash router (we don't yet use the mirror
    30  router). Moreover, it is easy to have a heuristic for when to buffer: only when
    31  some of the consumers are blocked and others aren't.
    32  
    33  The current implementation of the hash router is simple: it is a routine that is
    34  called directly from a processor, which hashes the relevant columns and calls
    35  `Push` on the correct consumer.
    36  
    37  The requirements for the new implementation are:
    38   - if at least one of the consumers is blocked, the router needs to continue
    39     absorbing rows, buffering rows for blocked consumers and sending rows to
    40     non-blocked consumers. This is required for preventing deadlocks.
    41   - if all consumers are blocked, the router must stop buffering rows. This is
    42     necessary to apply backpressure and prevent buffering a lot of rows when the
    43     producer is faster than the consumers.
    44  
    45  Note that the consumers we are concerned about here are `RowChannel`s which are
    46  implemented using go channels. Routers never send rows directly to gRPC (they go
    47  through a `RowChannel ` to an `outbox` goroutine which does gRPC calls).
    48  
    49  # Proposed design
    50  
    51  For a k-way hash router, create k goroutines and k `memRowContainer`s (later
    52  `diskRowContainer`s). Each goroutine is responsible for sending rows to a
    53  consumer.
    54  
    55  The main router routine adds rows to the containers and uses a channel or
    56  condition variable to wake up the goroutine. The goroutine `Push`es the first
    57  row (which blocks until it gets sent).
    58  
    59  To ensure the second requirement above, all the k goroutines as well as the main
    60  routine use a semaphore of capacity `k`. Whenever a goroutine has buffered rows,
    61  it acquires the semaphore; whenever it has no more buffered rows, it releases
    62  the semaphore. The main router routine tries to acquire the semaphore whenever
    63  it's trying to buffer a new row. The result here is that if all consumers have
    64  buffered rows, the router routine also blocks on the semaphore.
    65  
    66  ### Pros
    67  
    68   - Efficient when fanout is high and many consumers are blocked.
    69   - Efficient when no buffering is necessary (the goroutines will never acquire
    70     the semaphore in that case).
    71  
    72  ### Cons
    73  
    74   - Extra goroutines = extra overhead.
    75  
    76  ## Implementation notes
    77  
    78   - Proof-of-concept benchmarks showed very little difference between using a
    79     condition variable vs a wake-up channel.
    80   
    81   - Adding and removing rows to a `memRowContainer` has overhead (e.g. memory
    82     accounting). The implementation should use a small lookaside buffer to avoid
    83     going through the container if we only buffer a few rows.
    84  
    85   - The goroutine that sends the rows along should grab multiple buffered rows
    86     instead of reacquiring the mutex for each row.
    87  
    88   - The main routine can reduce overhead by only acquiring the semaphore
    89     occasionally (e.g. every 8 rows) - it's ok if we buffer a few extra rows
    90     before we block.
    91  
    92  # Considered alternatives
    93  
    94  ## Channels to k goroutines
    95  
    96  Similar to the proposed solution, except that the router routine sends rows to
    97  the goroutines via channels, and the goroutines are responsible for buffering as
    98  necessary.
    99  
   100  The goroutines have a loop which tries to either receive a row, or send a row to
   101  a consumer (via a `select`). This would require exposing the underlying channel
   102  (we can no longer hide it behind the `RowReceiver` interface).
   103  
   104  The solution still uses the semaphore but an optimization is possible: we can
   105  have a semaphore of `k-1` and only acquire the semaphore from the goroutines,
   106  the idea being that if all consumers are blocked, the last goroutine blocks,
   107  eventually causing the router routine to also block. This optimization has some
   108  subtleties (especially for k=2), and there are cases where it doesn't block as
   109  early as the proposed solution leading to more buffering (even when all
   110  consumers are blocked, the router routine will continue to send rows until it
   111  has to send to the one blocked goroutine (the last to acquire the semaphore).
   112  
   113  This solution seems more complicated to implement correctly, and
   114  proof-of-concept
   115  [benchmarks](https://github.com/RaduBerinde/playground/tree/master/buffering_router)
   116  suggest this solution (`Option1` in the benchmarks) is slower anyway.
   117  
   118  ## reflect.Select
   119  
   120  An alternative solution involves using the channels to the consumers directly
   121  and avoids the use of `k` goroutines.
   122  
   123  The hashing router routine receives a row destined to a certain consumer. If we
   124  don't have rows buffered for this consumer, we do a non-blocking send to the
   125  consumer. If that doesn't succeed, we buffer the row. In either case, if there
   126  are other consumers with buffered rows, we `TryPush` a row to each one
   127  (repeating if we are successful).
   128  
   129  If all the consumers have buffered rows, we need to block so we stop consuming
   130  more rows. We need to block until one consumer is able to receive a row; because
   131  the list of consumers is not fixed at compile time, we can't use a regular
   132  `select` statement; we would need to use `reflect.Select`.
   133  
   134  This solution was decided against because `reflect.Select` is likely [too
   135  slow](https://stackoverflow.com/a/32342741/4019276) and the solution is overall
   136  less idiomatic Go than the proposed solution.