Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

distsql: tpch query 7 results in context cancelled error #13989

Closed
asubiotto opened this issue Mar 8, 2017 · 24 comments · Fixed by #14103 or #14424
Closed

distsql: tpch query 7 results in context cancelled error #13989

asubiotto opened this issue Mar 8, 2017 · 24 comments · Fixed by #14103 or #14424
Assignees
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Milestone

Comments

@asubiotto
Copy link
Contributor

When running (execution plan)

SELECT
    supp_nation,
    cust_nation,
    l_year,
    SUM(volume) AS revenue
FROM
    (
        SELECT
            n1.n_name AS supp_nation,
            n2.n_name AS cust_nation,
            EXTRACT(year FROM l_shipdate) AS l_year,
            l_extendedprice * (1 - l_discount) AS volume
        FROM
            supplier,
            lineitem,
            orders,
            customer,
            nation n1,
            nation n2
        WHERE
            s_suppkey = l_suppkey
            AND o_orderkey = l_orderkey
            AND c_custkey = o_custkey
            AND s_nationkey = n1.n_nationkey
            AND c_nationkey = n2.n_nationkey
            AND (
                (n1.n_name = 'MOZAMBIQUE' AND n2.n_name = 'CANADA')
                or (n1.n_name = 'CANADA' AND n2.n_name = 'MOZAMBIQUE')
            )
            AND l_shipdate BETWEEN DATE '1995-01-01' AND DATE '1996-12-31'
    ) AS shipping
GROUP BY
    supp_nation,
    cust_nation,
    l_year
ORDER BY
    supp_nation,
    cust_nation,
    l_year;

against a 3-node local cluster sometimes crashes all three servers with

panic: too many Close() calls

goroutine 124280 [running]:
panic(0x5371480, 0xc428fe20b0)
    /usr/local/Cellar/go/1.7.4_2/libexec/src/runtime/panic.go:500 +0x1a1
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*MultiplexedRowChannel).Close(0xc43d2b8fc0, 0x5f41e60, 0xc458deca20)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/base.go:195 +0x21a
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*flowRegistry).finishInboundStreamLocked(0xc42023e020, 0xc4430d1520)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/flow_registry.go:274 +0xbf
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*flowRegistry).RegisterFlow.func1()
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/flow_registry.go:157 +0x2f6
created by time.goFunc
    /usr/local/Cellar/go/1.7.4_2/libexec/src/time/sleep.go:154 +0x44

And sometimes only crashes the gateway server with

panic: send on closed channel

goroutine 1280 [running]:
panic(0x53e93e0, 0xc439ee7400)
    /usr/local/Cellar/go/1.7.4_2/libexec/src/runtime/panic.go:500 +0x1a1
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*RowChannel).PushRow(0xc420225240, 0xc4202c43b8, 0x7, 0x7, 0x0)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/base.go:121 +0xc7
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*MultiplexedRowChannel).PushRow(0xc420225240, 0xc4202c43b8, 0x7, 0x7, 0xc449580000)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/base.go:181 +0x4b
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*procOutputHelper).emitRow(0xc4202c4388, 0x8316fb8, 0xc4211a4480, 0xc449580000, 0x7, 0x7, 0x5f60480)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/processors.go:197 +0x26c
github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*aggregator).Run(0xc4202c3200, 0x8316fb8, 0xc4211a4480, 0xc4201dede8)
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/aggregator.go:162 +0x432
created by github.com/cockroachdb/cockroach/pkg/sql/distsqlrun.(*Flow).Start
    /Users/asubiotto/go/src/github.com/cockroachdb/cockroach/pkg/sql/distsqlrun/flow.go:304 +0x329```
@asubiotto asubiotto added the C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. label Mar 8, 2017
@asubiotto asubiotto added this to the 1.0 milestone Mar 8, 2017
@asubiotto
Copy link
Contributor Author

This issue is caused by a flow not being set up in time and the corresponding inbound stream timing out (see flowRegistry.RegisterFlow). This timeout leads to a double close of the receiver which is an easy fix. However, the more interesting question is why the flow set up is timing out and if it is expected. Running the same query with the changes in #13570 produce a combination of successful runs and context canceled errors. The successful runs are due to changes to the flow set up code (changed to setting up ASAP rather than only after the first row is received) but the context canceled errors are worth digging into to determine whether a relation to flow set up timeouts exists.

This investigation will continue with the changes in #13570 rather than on master due to the size of the PR and the imminent merge.

Note: A simpler query to run to reproduce the same result is the inner SELECT.

@andreimatei
Copy link
Contributor

I've managed to reproduce a "context cancelled" error in another test that I'm looking at. Hopefully it's the same cause. Who knew we're so good at cancelling contexts?

@andreimatei
Copy link
Contributor

The "contex cancelled" error comes from a the server side of a grpc streaming connection returning from the RPC handler. The client's stream.Recv() call seems to return this "context cancelled" error, although the documentation seems to say that io.EOF should be returned. I need to play around with this... Currently the outbox expects an io.EOF from the server. Otherwise, the error is saved in outbox.Err and returned by RunSyncFlow.

Anyway, @asubiotto, if you see this error again, please give me a stack trace.

@RaduBerinde
Copy link
Member

Regardless, the code that crashes in finishInboundStreamLocked seems bogus; I think it just needs to be removed (we already close the receiver when we set timedOut):

	if is.timedOut {
 		is.receiver.Close(errors.Errorf("inbound stream timed out"))
 	}

vivekmenezes added a commit to vivekmenezes/cockroach that referenced this issue Mar 13, 2017
Close() is already called when timedOut is set.

fixes cockroachdb#13989
@andreimatei
Copy link
Contributor

andreimatei commented Mar 13, 2017 via email

@vivekmenezes
Copy link
Contributor

vivekmenezes commented Mar 13, 2017 via email

@andreimatei
Copy link
Contributor

Played around with gRPC a bit. The "context cancelled" errors that I was able to reproduce are received by a server that tries to access the stream after the rpc handler has returned (so a reference to the stream has been saved and used concurrently with the handler). But I'm not sure how that would happen with our production code, since servers (ProcessInboundStream) do not do async stuff.

So I'm not sure how the "context cancelled" error happened. When it happens again we need more info about which component is involved.

@vivekmenezes vivekmenezes reopened this Mar 14, 2017
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 14, 2017
We didn't have any tests exercising a stream failing to connect within
the registry's timeout.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 14, 2017
We didn't have any tests exercising a stream failing to connect within
the registry's timeout.

I've changed the fr.ConnectInbound interface slightly - made it return a
RowReceiver explicitly so that callers don't access the
inboundStreamInfo's fields without the flow registry lock - which is
documented to be required.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 14, 2017
We didn't have any tests exercising a stream failing to connect within
the registry's timeout.

I've changed the fr.ConnectInbound interface slightly - made it return a
RowReceiver explicitly so that callers don't access the
inboundStreamInfo's fields without the flow registry lock - which is
documented to be required.
@vivekmenezes
Copy link
Contributor

I'm certainly not seeing this any more on the nightly tests I'm running with schema changes. @asubiotto you were able to reproduce this. Can you still do?

@cuongdo
Copy link
Contributor

cuongdo commented Mar 15, 2017

He's on vacation. @arjunravinarayan do you see this issue when running your TPC-H load generator?

@rjnn
Copy link
Contributor

rjnn commented Mar 15, 2017 via email

@rjnn
Copy link
Contributor

rjnn commented Mar 15, 2017

Yes, I am still seeing this when running TPC-H query 7 on cluster omega (6 nodes, running beta candidate 10c2544) with distsql turned on.

Error occured when running query 7: pq: [n1] communication error: rpc error: code = 1 desc = context canceled
Finished query 7: 0 rows returned after 130.83 seconds

@rjnn
Copy link
Contributor

rjnn commented Mar 15, 2017

Apologies for taking so long, there were many bugs encountered along the way, including cockroachdb/loadgen#34, cockroachdb/loadgen#35, and some other discoveries in backup/restore.

@andreimatei
Copy link
Contributor

Great!
I'll grab you to show me how to run this fella so I can look into that error.

@andreimatei andreimatei changed the title distsql: tpch query 7 crashes servers distsql: tpch query 7 results in context cancelled error Mar 15, 2017
@vivekmenezes
Copy link
Contributor

@arjunravinarayan thanks! Glad you both are on it!

@andreimatei
Copy link
Contributor

I dug a little bit. I could repro this "context canceled" error on my laptop with 2 local nodes, although it doesn't always happen.

Based on the logs pasted below (taken from a node different from the gateway), this seems to be caused by the heartbeating that we do on gRPC connections to see if the connection is still alive. One of these heartbeats times out and so we close the gprc connection on the client side (the client here corresponds to the FlowStream RPC - so it's the data producer). This closing of the connection translates into a server-side "context canceled" error. It's interesting that the server receives the "context canceled" information (presumably a TPC reset packet) very soon after the client closes the connection, so communication at some level is not broken.

I have a theory that this is caused by bad multiplexing by gRPC of different calls on top of a single HTTP2 connection. Perhaps pushing a lot of data on a streaming RPC at the same time as trying to do a small heartbeat RPC doesn't guarantee any QoS for the heartbeat. I'll step in the lab to test this out.

E170317 15:59:26.016488 26 rpc/context.go:265  [n2] removing connection to localhost:26257 due to error: rpc error: code = 4 desc = context deadline exceeded
E170317 15:59:26.016661 5880 sql/distsqlrun/outbox.go:141  [n2] outbox flush error: rpc error: code = 13 desc = transport is closing
[unrelated transaction presumably using the same conn follows; notice that its heartbeat RPC (different kind of heartbeat) was also dog slow until it failed because of the conn closing]
W170317 15:59:26.016716 115 storage/node_liveness.go:293  [n2,hb] slow heartbeat took 5.2s
W170317 15:59:26.016741 115 storage/node_liveness.go:256  [n2,hb] failed node liveness heartbeat: result is ambiguous (sending to all 2 replicas failed; last error: rpc error: code = 13 desc = transport is closing, but RPC failure may have masked txn commit)

@andreimatei andreimatei self-assigned this Mar 17, 2017
@bdarnell
Copy link
Contributor

Ah, I see. Yeah, that argues for a higher ratio of connection quota to per-stream quota (and maybe a lower value for the per-stream quota, at least for these streams). At the lowest level HTTP/2 gives you a lot of flexibility about how to handle these quotas (for example, the connection quota need not be static; you could increase it whenever you find a lot of data buffered in streams with ordered synchronizers), but a lot of the subtleties are lost by the time you get up to the application layer.

@andreimatei
Copy link
Contributor

One mystery has been solved - we use @petermattis' fork of grpc which makes the conn window size = stream window size = 2MB. I checked for this fork before but I think I must have looked at the wrong file cause grpc is sub-vendored multiple times in our tree.

There is some WIP on grpc about dynamic connection window sizes that Tamir showed me: grpc/grpc-go#1126
I'm gonna look through it; unfortunately it doesn't seem to have documentation.
I'm not sure what the point of dynamic windows (other than accounting for variable "Bandwidth * Delay Products") would be, exactly. I suspect that this is the motivation of the grpc change above.

Ben, I'm not sure how serious you were about exploring different schemes. I think we do want to be cognizant of bandwidth limits and not thrash, but I don't think we want to use grpc flow control for that. We throttle replication of new ranges in/out of a node, we have some (primitive) admission control for DistSQL queries - I think these are better layers at which to deal with bandwidth. No?

andreimatei added a commit to cockroachdb/vendored that referenced this issue Mar 21, 2017
Switch from Peter's fork of grpc to my own fork, since Peter's out of
office. My fork is https://github.com/grpc/grpc-go head + Peter's one
commit increasing the per-stream flow-control window.

The motivation of syncing to newer gRPC is to include grpc/grpc-go#993
for helping with cockroachdb/cockroach#13989 -
we will move to gRPC internal heartbeats instead of using our own
connection heartbeats.

Other dep updates:
- the Lightstep update is a single commit that seems innocuous
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 21, 2017
Switch from Peter's fork of grpc to my own fork, since Peter's out of
office. My fork is https://github.com/grpc/grpc-go head + Peter's one
commit increasing the per-stream flow-control window.

The motivation of syncing to newer gRPC is to include grpc/grpc-go#993
for helping with cockroachdb#13989 - we will move to gRPC internal heartbeats
instead of using our own connection heartbeats in a future cockroach
commit.

Other dep updates:
- the Lightstep update is a single commit that seems innocuous
@bdarnell
Copy link
Contributor

One mystery has been solved - we use @petermattis' fork of grpc which makes the conn window size = stream window size = 2MB

Ah. Setting those two to equal is a bad idea since it lets one operation (like a snapshot) monopolize the whole connection.

I'm not sure what the point of dynamic windows (other than accounting for variable "Bandwidth * Delay Products") would be, exactly.

it lets you discover the BDP at runtime and use that to set the flow control limits instead of hard-coding a constant like we've done so far.

Ben, I'm not sure how serious you were about exploring different schemes. I think we do want to be cognizant of bandwidth limits and not thrash, but I don't think we want to use grpc flow control for that. We throttle replication of new ranges in/out of a node, we have some (primitive) admission control for DistSQL queries - I think these are better layers at which to deal with bandwidth. No?

I don't want to do anything too fancy here (i.e. I want to avoid dropping down to low-level HTTP/2 stuff). We should definitely tweak the numbers to set the connection limit higher than the stream limit (e.g. to 16x like it was before). If it's still an issue after that then we can look at making the per-connection limit unbounded and relying on other sorts of flow control

andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 26, 2017
- use gRPC Keepalives for detecting dead connections
Before this patch, the rpc.Context would perform heartbeats (a dedicated
tRPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble.
This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. Heartbeats stay for
the other current uses - clock skew detecting and node health marking.
Marking a node as unhealthy is debatable, give the shortcomings of these
RPCs. However, this marking currently doesn't have big consequences - it
only affects the order in which replicas are tried when a leaseholder is
unknown.

Fixes cockroachdb#13989

- make heartbeats and clock skew detecting optional. Not all users of
rpc.Context need this functionality.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 26, 2017
- use gRPC Keepalives for detecting dead connections
Before this patch, the rpc.Context would perform heartbeats (a dedicated
tRPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble.
This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. Heartbeats stay for
the other current uses - clock skew detecting and node health marking.
Marking a node as unhealthy is debatable, give the shortcomings of these
RPCs. However, this marking currently doesn't have big consequences - it
only affects the order in which replicas are tried when a leaseholder is
unknown.

Fixes cockroachdb#13989

- make heartbeats and clock skew detecting optional. Not all users of
rpc.Context need this functionality.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 28, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Mar 30, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 4, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 6, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 6, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 7, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 7, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
andreimatei added a commit to andreimatei/cockroach that referenced this issue Apr 7, 2017
Fixes cockroachdb#13989

Before this patch, the rpc.Context would perform heartbeats (a dedicated
RPC) to see if a connection is healthy. If the heartbeats failed, the
connection was closed (causing in-flight RPCs to fail) and the node was
marked as unhealthy.
These heartbeats, being regular RPCs, were subject to gRPC's flow
control. This means that they were easily blocked by other large RPCs,
which meant they were too feeble. In particular, they were easily
blocked by large DistSQL streams.

This patch moves to using gRPC's internal HTTP2 ping frames for checking
conn health. These are not subject to flow control. The grpc
transport-level connection is closed when they fail (and so in-flight
RPCs still fail), but otherwise gRPC reconnects transparently.
Heartbeats stay for the other current uses - clock skew detection and
node health marking. Marking a node as unhealthy is debatable, give the
shortcomings of these RPCs. However, this marking currently doesn't have
big consequences - it only affects the order in which replicas are tried
when a leaseholder is unknown.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior.
Projects
None yet
8 participants