2013-12-07 00:54:33 +00:00
package consul
import (
2020-06-24 16:36:54 +00:00
"context"
2014-04-04 23:34:23 +00:00
"crypto/tls"
2020-03-09 20:59:02 +00:00
"encoding/binary"
2019-07-24 21:06:39 +00:00
"errors"
2013-12-11 22:04:44 +00:00
"fmt"
2013-12-11 22:57:40 +00:00
"io"
2013-12-07 00:54:33 +00:00
"net"
2014-02-05 23:29:52 +00:00
"strings"
2020-02-10 18:01:15 +00:00
"sync/atomic"
2014-02-05 18:38:29 +00:00
"time"
2014-11-26 09:25:37 +00:00
"github.com/armon/go-metrics"
2020-11-13 02:12:12 +00:00
"github.com/armon/go-metrics/prometheus"
2020-01-31 16:19:37 +00:00
connlimit "github.com/hashicorp/go-connlimit"
2020-01-28 23:50:41 +00:00
"github.com/hashicorp/go-hclog"
2017-07-06 10:48:37 +00:00
memdb "github.com/hashicorp/go-memdb"
2019-07-24 21:06:39 +00:00
"github.com/hashicorp/go-raftchunking"
2015-12-16 19:38:35 +00:00
"github.com/hashicorp/memberlist"
2019-03-06 17:13:28 +00:00
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
2019-07-24 21:06:39 +00:00
"github.com/hashicorp/raft"
2014-11-26 09:25:37 +00:00
"github.com/hashicorp/yamux"
2021-09-22 18:14:26 +00:00
"google.golang.org/grpc"
2021-04-08 22:58:15 +00:00
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/consul/wanfed"
"github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/pool"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/logging"
2013-12-07 00:54:33 +00:00
)
2020-11-13 02:12:12 +00:00
var RPCCounters = [ ] prometheus . CounterDefinition {
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "accept_conn" } ,
2020-11-16 19:02:11 +00:00
Help : "Increments when a server accepts an RPC connection." ,
2020-11-13 02:12:12 +00:00
} ,
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "raft_handoff" } ,
2020-11-16 19:02:11 +00:00
Help : "Increments when a server accepts a Raft-related RPC connection." ,
2020-11-13 02:12:12 +00:00
} ,
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "request_error" } ,
2020-11-16 19:02:11 +00:00
Help : "Increments when a server returns an error from an RPC request." ,
2020-11-13 02:12:12 +00:00
} ,
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "request" } ,
2020-11-16 19:02:11 +00:00
Help : "Increments when a server receives a Consul-related RPC request." ,
2020-11-13 02:12:12 +00:00
} ,
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "cross-dc" } ,
2020-11-16 19:02:11 +00:00
Help : "Increments when a server sends a (potentially blocking) cross datacenter RPC query." ,
2020-11-13 02:12:12 +00:00
} ,
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "query" } ,
2021-06-25 20:37:26 +00:00
Help : "Increments when a server receives a read request, indicating the rate of new read queries." ,
2020-11-13 02:12:12 +00:00
} ,
}
var RPCGauges = [ ] prometheus . GaugeDefinition {
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "queries_blocking" } ,
2020-11-16 19:02:11 +00:00
Help : "Shows the current number of in-flight blocking queries the server is handling." ,
2020-11-13 02:12:12 +00:00
} ,
}
var RPCSummaries = [ ] prometheus . SummaryDefinition {
{
2020-11-13 21:18:04 +00:00
Name : [ ] string { "rpc" , "consistentRead" } ,
2020-11-16 19:02:11 +00:00
Help : "Measures the time spent confirming that a consistent read can be performed." ,
2020-11-13 02:12:12 +00:00
} ,
}
2014-02-05 18:38:29 +00:00
const (
2014-04-29 05:25:09 +00:00
// Warn if the Raft command is larger than this.
2014-05-06 21:10:08 +00:00
// If it's over 1MB something is probably being abusive.
raftWarnSize = 1024 * 1024
2014-05-10 02:16:40 +00:00
// enqueueLimit caps how long we will wait to enqueue
// a new Raft command. Something is probably wrong if this
// value is ever reached. However, it prevents us from blocking
// the requesting goroutine forever.
enqueueLimit = 30 * time . Second
2014-02-05 18:38:29 +00:00
)
2021-04-20 18:23:50 +00:00
var ErrChunkingResubmit = errors . New ( "please resubmit call for rechunking" )
2019-07-24 21:06:39 +00:00
2020-01-28 23:50:41 +00:00
func ( s * Server ) rpcLogger ( ) hclog . Logger {
return s . loggers . Named ( logging . RPC )
}
2013-12-07 00:54:33 +00:00
// listen is used to listen for incoming RPC connections
2017-08-29 00:58:22 +00:00
func ( s * Server ) listen ( listener net . Listener ) {
2013-12-07 00:54:33 +00:00
for {
// Accept a connection
2017-08-29 00:58:22 +00:00
conn , err := listener . Accept ( )
2013-12-07 00:54:33 +00:00
if err != nil {
if s . shutdown {
return
}
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "failed to accept RPC conn" , "error" , err )
2013-12-07 00:54:33 +00:00
continue
}
2020-01-31 16:19:37 +00:00
free , err := s . rpcConnLimiter . Accept ( conn )
if err != nil {
2020-02-12 15:37:16 +00:00
s . rpcLogger ( ) . Error ( "rejecting RPC conn from because rpc_max_conns_per_client exceeded" , "conn" , logConn ( conn ) )
2020-01-31 16:19:37 +00:00
conn . Close ( )
continue
}
// Wrap conn so it will be auto-freed from conn limiter when it closes.
conn = connlimit . Wrap ( conn , free )
2014-04-07 19:45:33 +00:00
go s . handleConn ( conn , false )
2017-10-04 23:43:27 +00:00
metrics . IncrCounter ( [ ] string { "rpc" , "accept_conn" } , 1 )
2013-12-07 00:54:33 +00:00
}
}
2015-12-16 19:38:35 +00:00
// logConn is a wrapper around memberlist's LogConn so that we format references
// to "from" addresses in a consistent way. This is just a shorter name.
func logConn ( conn net . Conn ) string {
return memberlist . LogConn ( conn )
}
2013-12-09 21:13:40 +00:00
// handleConn is used to determine if this is a Raft or
// Consul type RPC connection and invoke the correct handler
2014-04-07 19:45:33 +00:00
func ( s * Server ) handleConn ( conn net . Conn , isTLS bool ) {
2020-01-31 16:19:37 +00:00
// Limit how long the client can hold the connection open before they send the
// magic byte (and authenticate when mTLS is enabled). If `isTLS == true` then
// this also enforces a timeout on how long it takes for the handshake to
// complete since tls.Conn.Read implicitly calls Handshake().
if s . config . RPCHandshakeTimeout > 0 {
conn . SetReadDeadline ( time . Now ( ) . Add ( s . config . RPCHandshakeTimeout ) )
}
2020-03-09 20:59:02 +00:00
if ! isTLS && s . tlsConfigurator . MutualTLSCapable ( ) {
// See if actually this is native TLS multiplexed onto the old
// "type-byte" system.
peekedConn , nativeTLS , err := pool . PeekForTLS ( conn )
if err != nil {
if err != io . EOF {
s . rpcLogger ( ) . Error (
"failed to read first byte" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
}
conn . Close ( )
return
}
if nativeTLS {
s . handleNativeTLS ( peekedConn )
return
}
conn = peekedConn
}
// Read a single byte
buf := make ( [ ] byte , 1 )
2013-12-09 21:13:40 +00:00
if _ , err := conn . Read ( buf ) ; err != nil {
2014-05-23 23:28:55 +00:00
if err != io . EOF {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "failed to read byte" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
2014-05-23 23:28:55 +00:00
}
2013-12-09 21:13:40 +00:00
conn . Close ( )
return
}
2017-06-15 13:16:16 +00:00
typ := pool . RPCType ( buf [ 0 ] )
2013-12-09 21:13:40 +00:00
2020-01-31 16:19:37 +00:00
// Reset the deadline as we aren't sure what is expected next - it depends on
// the protocol.
if s . config . RPCHandshakeTimeout > 0 {
conn . SetReadDeadline ( time . Time { } )
}
2014-04-04 23:43:00 +00:00
// Enforce TLS if VerifyIncoming is set
2019-06-27 20:22:07 +00:00
if s . tlsConfigurator . VerifyIncomingRPC ( ) && ! isTLS && typ != pool . RPCTLS && typ != pool . RPCTLSInsecure {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Warn ( "Non-TLS connection attempted with VerifyIncoming set" , "conn" , logConn ( conn ) )
2014-04-04 23:43:00 +00:00
conn . Close ( )
return
}
2013-12-09 21:13:40 +00:00
// Switch on the byte
2017-06-15 13:16:16 +00:00
switch typ {
case pool . RPCConsul :
2013-12-09 21:13:40 +00:00
s . handleConsulConn ( conn )
2017-06-15 13:16:16 +00:00
case pool . RPCRaft :
2021-08-26 22:04:32 +00:00
s . handleRaftRPC ( conn )
2013-12-09 21:13:40 +00:00
2017-06-15 13:16:16 +00:00
case pool . RPCTLS :
2020-01-31 16:19:37 +00:00
// Don't allow malicious client to create TLS-in-TLS for ever.
if isTLS {
2020-02-12 15:37:16 +00:00
s . rpcLogger ( ) . Error ( "TLS connection attempting to establish inner TLS connection" , "conn" , logConn ( conn ) )
2020-01-31 16:19:37 +00:00
conn . Close ( )
return
}
2019-06-27 20:22:07 +00:00
conn = tls . Server ( conn , s . tlsConfigurator . IncomingRPCConfig ( ) )
2014-04-07 19:45:33 +00:00
s . handleConn ( conn , true )
2017-06-15 13:16:16 +00:00
case pool . RPCMultiplexV2 :
2014-05-27 18:00:35 +00:00
s . handleMultiplexV2 ( conn )
2017-06-15 13:16:16 +00:00
case pool . RPCSnapshot :
2016-10-26 02:20:24 +00:00
s . handleSnapshotConn ( conn )
2019-06-27 20:22:07 +00:00
case pool . RPCTLSInsecure :
2020-01-31 16:19:37 +00:00
// Don't allow malicious client to create TLS-in-TLS for ever.
if isTLS {
2020-02-12 15:37:16 +00:00
s . rpcLogger ( ) . Error ( "TLS connection attempting to establish inner TLS connection" , "conn" , logConn ( conn ) )
2020-01-31 16:19:37 +00:00
conn . Close ( )
return
}
2019-06-27 20:22:07 +00:00
conn = tls . Server ( conn , s . tlsConfigurator . IncomingInsecureRPCConfig ( ) )
s . handleInsecureConn ( conn )
2020-07-22 23:57:29 +00:00
case pool . RPCGRPC :
s . grpcHandler . Handle ( conn )
2013-12-09 21:13:40 +00:00
default :
2018-05-24 14:36:42 +00:00
if ! s . handleEnterpriseRPCConn ( typ , conn , isTLS ) {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "unrecognized RPC byte" ,
"byte" , typ ,
"conn" , logConn ( conn ) ,
)
2018-05-24 14:36:42 +00:00
conn . Close ( )
}
2013-12-09 21:13:40 +00:00
}
}
2020-03-09 20:59:02 +00:00
func ( s * Server ) handleNativeTLS ( conn net . Conn ) {
s . rpcLogger ( ) . Trace (
"detected actual TLS over RPC port" ,
"conn" , logConn ( conn ) ,
)
tlscfg := s . tlsConfigurator . IncomingALPNRPCConfig ( pool . RPCNextProtos )
tlsConn := tls . Server ( conn , tlscfg )
// Force the handshake to conclude.
if err := tlsConn . Handshake ( ) ; err != nil {
s . rpcLogger ( ) . Error (
"TLS handshake failed" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
conn . Close ( )
return
}
// Reset the deadline as we aren't sure what is expected next - it depends on
// the protocol.
if s . config . RPCHandshakeTimeout > 0 {
conn . SetReadDeadline ( time . Time { } )
}
var (
cs = tlsConn . ConnectionState ( )
sni = cs . ServerName
nextProto = cs . NegotiatedProtocol
transport = s . memberlistTransportWAN
)
s . rpcLogger ( ) . Trace (
"accepted nativeTLS RPC" ,
"sni" , sni ,
"protocol" , nextProto ,
"conn" , logConn ( conn ) ,
)
switch nextProto {
case pool . ALPN_RPCConsul :
s . handleConsulConn ( tlsConn )
case pool . ALPN_RPCRaft :
2021-08-26 22:04:32 +00:00
s . handleRaftRPC ( tlsConn )
2020-03-09 20:59:02 +00:00
case pool . ALPN_RPCMultiplexV2 :
s . handleMultiplexV2 ( tlsConn )
case pool . ALPN_RPCSnapshot :
s . handleSnapshotConn ( tlsConn )
2020-07-22 23:57:29 +00:00
case pool . ALPN_RPCGRPC :
2021-08-24 21:28:44 +00:00
s . grpcHandler . Handle ( tlsConn )
2020-07-22 23:57:29 +00:00
2020-03-09 20:59:02 +00:00
case pool . ALPN_WANGossipPacket :
if err := s . handleALPN_WANGossipPacketStream ( tlsConn ) ; err != nil && err != io . EOF {
s . rpcLogger ( ) . Error (
"failed to ingest RPC" ,
"sni" , sni ,
"protocol" , nextProto ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
}
case pool . ALPN_WANGossipStream :
// No need to defer the conn.Close() here, the Ingest methods do that.
if err := transport . IngestStream ( tlsConn ) ; err != nil {
s . rpcLogger ( ) . Error (
"failed to ingest RPC" ,
"sni" , sni ,
"protocol" , nextProto ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
}
default :
if ! s . handleEnterpriseNativeTLSConn ( nextProto , conn ) {
s . rpcLogger ( ) . Error (
"discarding RPC for unknown negotiated protocol" ,
"failed to ingest RPC" ,
"protocol" , nextProto ,
"conn" , logConn ( conn ) ,
)
conn . Close ( )
}
}
}
2014-05-27 18:00:35 +00:00
// handleMultiplexV2 is used to multiplex a single incoming connection
// using the Yamux multiplexer
func ( s * Server ) handleMultiplexV2 ( conn net . Conn ) {
defer conn . Close ( )
2014-05-28 23:32:10 +00:00
conf := yamux . DefaultConfig ( )
2020-07-29 18:43:33 +00:00
// override the default because LogOutput conflicts with Logger
conf . LogOutput = nil
// TODO: should this be created once and cached?
conf . Logger = s . logger . StandardLogger ( & hclog . StandardLoggerOptions { InferLevels : true } )
2014-05-28 23:32:10 +00:00
server , _ := yamux . Server ( conn , conf )
2014-05-27 18:00:35 +00:00
for {
sub , err := server . Accept ( )
if err != nil {
if err != io . EOF {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "multiplex conn accept failed" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
2014-05-27 18:00:35 +00:00
}
return
}
2020-04-30 20:12:17 +00:00
// In the beginning only RPC was supposed to be multiplexed
// with yamux. In order to add the ability to multiplex network
// area connections, this workaround was added.
// This code peeks the first byte and checks if it is
// RPCGossip, in which case this is handled by enterprise code.
// Otherwise this connection is handled like before by the RPC
// handler.
// This wouldn't work if a normal RPC could start with
// RPCGossip(6). In messagepack a 6 encodes a positive fixint:
// https://github.com/msgpack/msgpack/blob/master/spec.md.
// None of the RPCs we are doing starts with that, usually it is
// a string for datacenter.
peeked , first , err := pool . PeekFirstByte ( sub )
if err != nil {
s . rpcLogger ( ) . Error ( "Problem peeking connection" , "conn" , logConn ( sub ) , "err" , err )
sub . Close ( )
return
}
sub = peeked
switch first {
2021-08-24 21:28:44 +00:00
case byte ( pool . RPCGossip ) :
2020-04-30 20:12:17 +00:00
buf := make ( [ ] byte , 1 )
sub . Read ( buf )
go func ( ) {
if ! s . handleEnterpriseRPCConn ( pool . RPCGossip , sub , false ) {
s . rpcLogger ( ) . Error ( "unrecognized RPC byte" ,
"byte" , pool . RPCGossip ,
"conn" , logConn ( conn ) ,
)
sub . Close ( )
}
} ( )
default :
go s . handleConsulConn ( sub )
}
2014-05-27 18:00:35 +00:00
}
}
2013-12-09 21:13:40 +00:00
// handleConsulConn is used to service a single Consul RPC connection
func ( s * Server ) handleConsulConn ( conn net . Conn ) {
2014-05-15 00:34:24 +00:00
defer conn . Close ( )
2020-02-07 21:50:24 +00:00
rpcCodec := msgpackrpc . NewCodecFromHandle ( true , true , conn , structs . MsgpackHandle )
2014-11-26 09:25:37 +00:00
for {
select {
case <- s . shutdownCh :
return
default :
}
2013-12-07 00:54:33 +00:00
if err := s . rpcServer . ServeRequest ( rpcCodec ) ; err != nil {
2014-05-28 23:28:06 +00:00
if err != io . EOF && ! strings . Contains ( err . Error ( ) , "closed" ) {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "RPC error" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
2017-10-04 23:43:27 +00:00
metrics . IncrCounter ( [ ] string { "rpc" , "request_error" } , 1 )
2013-12-11 22:57:40 +00:00
}
2013-12-07 00:54:33 +00:00
return
}
2017-10-04 23:43:27 +00:00
metrics . IncrCounter ( [ ] string { "rpc" , "request" } , 1 )
2013-12-07 00:54:33 +00:00
}
}
2013-12-11 22:04:44 +00:00
2019-06-27 20:22:07 +00:00
// handleInsecureConsulConn is used to service a single Consul INSECURERPC connection
func ( s * Server ) handleInsecureConn ( conn net . Conn ) {
defer conn . Close ( )
2020-02-07 21:50:24 +00:00
rpcCodec := msgpackrpc . NewCodecFromHandle ( true , true , conn , structs . MsgpackHandle )
2019-06-27 20:22:07 +00:00
for {
select {
case <- s . shutdownCh :
return
default :
}
if err := s . insecureRPCServer . ServeRequest ( rpcCodec ) ; err != nil {
if err != io . EOF && ! strings . Contains ( err . Error ( ) , "closed" ) {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "INSECURERPC error" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
2019-06-27 20:22:07 +00:00
metrics . IncrCounter ( [ ] string { "rpc" , "request_error" } , 1 )
}
return
}
metrics . IncrCounter ( [ ] string { "rpc" , "request" } , 1 )
}
}
2016-10-26 02:20:24 +00:00
// handleSnapshotConn is used to dispatch snapshot saves and restores, which
// stream so don't use the normal RPC mechanism.
func ( s * Server ) handleSnapshotConn ( conn net . Conn ) {
go func ( ) {
defer conn . Close ( )
if err := s . handleSnapshotRequest ( conn ) ; err != nil {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "Snapshot RPC error" ,
"conn" , logConn ( conn ) ,
"error" , err ,
)
2016-10-26 02:20:24 +00:00
}
} ( )
}
2021-08-26 22:04:32 +00:00
func ( s * Server ) handleRaftRPC ( conn net . Conn ) {
if tlsConn , ok := conn . ( * tls . Conn ) ; ok {
err := s . tlsConfigurator . AuthorizeServerConn ( s . config . Datacenter , tlsConn )
if err != nil {
s . rpcLogger ( ) . Warn ( err . Error ( ) , "from" , conn . RemoteAddr ( ) , "operation" , "raft RPC" )
conn . Close ( )
return
}
}
metrics . IncrCounter ( [ ] string { "rpc" , "raft_handoff" } , 1 )
s . raftLayer . Handoff ( conn )
}
2020-03-09 20:59:02 +00:00
func ( s * Server ) handleALPN_WANGossipPacketStream ( conn net . Conn ) error {
defer conn . Close ( )
transport := s . memberlistTransportWAN
for {
select {
case <- s . shutdownCh :
return nil
default :
}
// Note: if we need to change this format to have additional header
// information we can just negotiate a different ALPN protocol instead
// of needing any sort of version field here.
prefixLen , err := readUint32 ( conn , wanfed . GossipPacketMaxIdleTime )
if err != nil {
return err
}
// Avoid a memory exhaustion DOS vector here by capping how large this
// packet can be to something reasonable.
if prefixLen > wanfed . GossipPacketMaxByteSize {
return fmt . Errorf ( "gossip packet size %d exceeds threshold of %d" , prefixLen , wanfed . GossipPacketMaxByteSize )
}
lc := & limitedConn {
Conn : conn ,
lr : io . LimitReader ( conn , int64 ( prefixLen ) ) ,
}
if err := transport . IngestPacket ( lc , conn . RemoteAddr ( ) , time . Now ( ) , false ) ; err != nil {
return err
}
}
}
func readUint32 ( conn net . Conn , timeout time . Duration ) ( uint32 , error ) {
// Since requests are framed we can easily just set a deadline on
// reading that frame and then disable it for the rest of the body.
if err := conn . SetReadDeadline ( time . Now ( ) . Add ( timeout ) ) ; err != nil {
return 0 , err
}
var v uint32
if err := binary . Read ( conn , binary . BigEndian , & v ) ; err != nil {
return 0 , err
}
if err := conn . SetReadDeadline ( time . Time { } ) ; err != nil {
return 0 , err
}
return v , nil
}
type limitedConn struct {
net . Conn
lr io . Reader
}
func ( c * limitedConn ) Read ( b [ ] byte ) ( n int , err error ) {
return c . lr . Read ( b )
}
2021-04-20 18:23:50 +00:00
// canRetry returns true if the request and error indicate that a retry is safe.
2021-05-27 21:29:43 +00:00
func canRetry ( info structs . RPCInfo , err error , start time . Time , config * Config ) bool {
if info != nil && info . HasTimedOut ( start , config . RPCHoldTimeout , config . MaxQueryTime , config . DefaultQueryTime ) {
// RPCInfo timeout may include extra time for MaxQueryTime
return false
} else if info == nil && time . Since ( start ) > config . RPCHoldTimeout {
// When not RPCInfo, timeout is only RPCHoldTimeout
return false
}
2017-10-10 22:19:50 +00:00
// No leader errors are always safe to retry since no state could have
// been changed.
if structs . IsErrNoLeader ( err ) {
return true
}
2021-04-09 17:34:09 +00:00
// If we are chunking and it doesn't seem to have completed, try again.
if err != nil && strings . Contains ( err . Error ( ) , ErrChunkingResubmit . Error ( ) ) {
2019-07-24 21:06:39 +00:00
return true
}
2017-10-10 22:19:50 +00:00
// Reads are safe to retry for stream errors, such as if a server was
// being shut down.
2021-04-20 18:23:50 +00:00
return info != nil && info . IsRead ( ) && lib . IsErrEOF ( err )
2017-10-10 22:19:50 +00:00
}
2021-09-22 18:14:26 +00:00
// ForwardRPC is used to potentially forward an RPC request to a remote DC or
// to the local leader depending upon the request.
//
// Returns a bool of if forwarding was performed, as well as any error. If
// false is returned (with no error) it is assumed that the current server
// should handle the request.
2021-05-27 21:29:43 +00:00
func ( s * Server ) ForwardRPC ( method string , info structs . RPCInfo , reply interface { } ) ( bool , error ) {
2021-09-22 18:14:26 +00:00
forwardToDC := func ( dc string ) error {
return s . forwardDC ( method , dc , info , reply )
}
forwardToLeader := func ( leader * metadata . Server ) error {
return s . connPool . RPC ( s . config . Datacenter , leader . ShortName , leader . Addr ,
method , info , reply )
}
return s . forwardRPC ( info , forwardToDC , forwardToLeader )
}
// ForwardGRPC is used to potentially forward an RPC request to a remote DC or
// to the local leader depending upon the request.
//
// Returns a bool of if forwarding was performed, as well as any error. If
// false is returned (with no error) it is assumed that the current server
// should handle the request.
func ( s * Server ) ForwardGRPC ( connPool GRPCClientConner , info structs . RPCInfo , f func ( * grpc . ClientConn ) error ) ( handled bool , err error ) {
forwardToDC := func ( dc string ) error {
conn , err := connPool . ClientConn ( dc )
if err != nil {
return err
}
return f ( conn )
}
forwardToLeader := func ( leader * metadata . Server ) error {
conn , err := connPool . ClientConnLeader ( )
if err != nil {
return err
}
return f ( conn )
}
return s . forwardRPC ( info , forwardToDC , forwardToLeader )
}
2016-07-10 17:24:06 +00:00
2021-09-22 18:14:26 +00:00
// forwardRPC is used to potentially forward an RPC request to a remote DC or
// to the local leader depending upon the request.
//
// If info.RequestDatacenter() does not match the local datacenter, then the
// request will be forwarded to the DC using forwardToDC.
//
// Stale read requests will be handled locally if the current node has an
// initialized raft database, otherwise requests will be forwarded to the local
// leader using forwardToLeader.
//
// Returns a bool of if forwarding was performed, as well as any error. If
// false is returned (with no error) it is assumed that the current server
// should handle the request.
func ( s * Server ) forwardRPC (
info structs . RPCInfo ,
forwardToDC func ( dc string ) error ,
forwardToLeader func ( leader * metadata . Server ) error ,
) ( handled bool , err error ) {
// Forward the request to the requested datacenter.
if handled , err := s . forwardRequestToOtherDatacenter ( info , forwardToDC ) ; handled || err != nil {
return handled , err
}
// See if we should let this server handle the read request without
// shipping the request to the leader.
if s . canServeReadRequest ( info ) {
return false , nil
}
return s . forwardRequestToLeader ( info , forwardToLeader )
}
// forwardRequestToOtherDatacenter is an implementation detail of forwardRPC.
// See the comment for forwardRPC for more details.
func ( s * Server ) forwardRequestToOtherDatacenter ( info structs . RPCInfo , forwardToDC func ( dc string ) error ) ( handled bool , err error ) {
2013-12-11 22:04:44 +00:00
// Handle DC forwarding
2021-05-27 21:29:43 +00:00
dc := info . RequestDatacenter ( )
2021-09-22 18:14:26 +00:00
if dc == "" {
dc = s . config . Datacenter
}
2013-12-11 22:04:44 +00:00
if dc != s . config . Datacenter {
2020-03-10 16:15:22 +00:00
// Local tokens only work within the current datacenter. Check to see
// if we are attempting to forward one to a remote datacenter and strip
// it, falling back on the anonymous token on the other end.
2021-05-27 21:29:43 +00:00
if token := info . TokenSecret ( ) ; token != "" {
2020-03-10 16:15:22 +00:00
done , ident , err := s . ResolveIdentityFromToken ( token )
if done {
if err != nil && ! acl . IsErrNotFound ( err ) {
return false , err
}
if ident != nil && ident . IsLocal ( ) {
// Strip it from the request.
2021-05-27 21:29:43 +00:00
info . SetTokenSecret ( "" )
defer info . SetTokenSecret ( token )
2020-03-10 16:15:22 +00:00
}
}
}
2021-09-22 18:14:26 +00:00
return true , forwardToDC ( dc )
2013-12-11 22:04:44 +00:00
}
2021-09-22 18:14:26 +00:00
return false , nil
}
// canServeReadRequest determines if the request is a stale read request and
// the current node can safely process that request.
func ( s * Server ) canServeReadRequest ( info structs . RPCInfo ) bool {
2018-08-23 16:06:39 +00:00
// Check if we can allow a stale read, ensure our local DB is initialized
2021-09-22 18:14:26 +00:00
return info . IsRead ( ) && info . AllowStaleRead ( ) && ! s . raft . LastContact ( ) . IsZero ( )
}
2014-04-19 00:26:59 +00:00
2021-09-22 18:14:26 +00:00
// forwardRequestToLeader is an implementation detail of forwardRPC.
// See the comment for forwardRPC for more details.
func ( s * Server ) forwardRequestToLeader ( info structs . RPCInfo , forwardToLeader func ( leader * metadata . Server ) error ) ( handled bool , err error ) {
firstCheck := time . Now ( )
2016-07-10 17:24:06 +00:00
CHECK_LEADER :
2017-10-10 22:19:50 +00:00
// Fail fast if we are in the process of leaving
select {
case <- s . leaveCh :
return true , structs . ErrNoLeader
default :
}
2016-07-10 17:24:06 +00:00
// Find the leader
2021-01-04 19:05:23 +00:00
isLeader , leader , rpcErr := s . getLeader ( )
2016-07-10 17:24:06 +00:00
// Handle the case we are the leader
if isLeader {
return false , nil
}
// Handle the case of a known leader
2017-10-10 22:19:50 +00:00
if leader != nil {
2021-09-22 18:14:26 +00:00
rpcErr = forwardToLeader ( leader )
2021-05-27 21:29:43 +00:00
if rpcErr == nil {
return true , nil
2017-10-10 22:19:50 +00:00
}
2013-12-11 22:04:44 +00:00
}
2016-07-10 17:24:06 +00:00
2021-05-27 21:29:43 +00:00
if retry := canRetry ( info , rpcErr , firstCheck , s . config ) ; retry {
// Gate the request until there is a leader
jitter := lib . RandomStagger ( s . config . RPCHoldTimeout / structs . JitterFraction )
2016-07-10 17:24:06 +00:00
select {
case <- time . After ( jitter ) :
goto CHECK_LEADER
2017-10-10 22:19:50 +00:00
case <- s . leaveCh :
2016-07-10 17:24:06 +00:00
case <- s . shutdownCh :
}
}
// No leader found and hold time exceeded
2017-10-10 22:19:50 +00:00
return true , rpcErr
2013-12-11 22:04:44 +00:00
}
2016-10-26 02:20:24 +00:00
// getLeader returns if the current node is the leader, and if not then it
// returns the leader which is potentially nil if the cluster has not yet
2021-01-04 19:05:23 +00:00
// elected a leader. In the case of not having a leader elected yet
// then a NoClusterLeader error gets returned. In the case of Raft having
// a leader but out internal tracking failing to find the leader we
// return a LeaderNotTracked error. Therefore if the err is nil AND
// the bool is false then the Server will be non-nil
func ( s * Server ) getLeader ( ) ( bool , * metadata . Server , error ) {
2016-07-10 17:24:06 +00:00
// Check if we are the leader
if s . IsLeader ( ) {
2021-01-04 19:05:23 +00:00
return true , nil , nil
2016-07-10 17:24:06 +00:00
}
2014-05-27 22:45:19 +00:00
// Get the leader
2013-12-11 22:04:44 +00:00
leader := s . raft . Leader ( )
2015-05-08 18:35:12 +00:00
if leader == "" {
2021-01-04 19:05:23 +00:00
return false , nil , structs . ErrNoLeader
2013-12-11 22:04:44 +00:00
}
2014-05-27 22:45:19 +00:00
// Lookup the server
2017-08-30 17:31:36 +00:00
server := s . serverLookup . Server ( leader )
2014-05-27 22:45:19 +00:00
2021-01-04 19:05:23 +00:00
// if server is nil this indicates that while we have a Raft leader
// something has caused that node to be considered unhealthy which
// cascades into its removal from the serverLookup struct. In this case
// we should not report no cluster leader but instead report a different
// error so as not to confuse our users as to the what the root cause of
// an issue might be.
if server == nil {
s . logger . Warn ( "Raft has a leader but other tracking of the node would indicate that the node is unhealthy or does not exist. The network may be misconfigured." , "leader" , leader )
return false , nil , structs . ErrLeaderNotTracked
}
return false , server , nil
2016-07-10 17:24:06 +00:00
}
2016-10-26 02:20:24 +00:00
// forwardDC is used to forward an RPC call to a remote DC, or fail if no servers
func ( s * Server ) forwardDC ( method , dc string , args interface { } , reply interface { } ) error {
2017-03-14 01:54:34 +00:00
manager , server , ok := s . router . FindRoute ( dc )
2016-10-26 02:20:24 +00:00
if ! ok {
2019-09-03 15:46:24 +00:00
if s . router . HasDatacenter ( dc ) {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Warn ( "RPC request to DC is currently failing as no server can be reached" , "datacenter" , dc )
2019-09-03 15:46:24 +00:00
return structs . ErrDCNotAvailable
}
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Warn ( "RPC request for DC is currently failing as no path was found" ,
"datacenter" , dc ,
"method" , method ,
)
2016-10-26 02:20:24 +00:00
return structs . ErrNoDCPath
}
2013-12-12 00:33:19 +00:00
2017-10-04 23:43:27 +00:00
metrics . IncrCounterWithLabels ( [ ] string { "rpc" , "cross-dc" } , 1 ,
[ ] metrics . Label { { Name : "datacenter" , Value : dc } } )
2020-05-28 07:48:34 +00:00
if err := s . connPool . RPC ( dc , server . ShortName , server . Addr , method , args , reply ) ; err != nil {
2017-03-14 01:54:34 +00:00
manager . NotifyFailedServer ( server )
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Error ( "RPC failed to server in DC" ,
"server" , server . Addr ,
"datacenter" , dc ,
"method" , method ,
"error" , err ,
)
2017-03-14 01:54:34 +00:00
return err
}
return nil
2013-12-11 22:04:44 +00:00
}
2020-08-11 11:35:48 +00:00
// keyringRPCs is used to forward an RPC request to a server in each dc. This
// will only error for RPC-related errors. Otherwise, application-level errors
// can be sent in the response objects.
func ( s * Server ) keyringRPCs ( method string , args interface { } , dcs [ ] string ) ( * structs . KeyringResponses , error ) {
2014-10-02 06:09:00 +00:00
2020-08-11 11:35:48 +00:00
errorCh := make ( chan error , len ( dcs ) )
respCh := make ( chan * structs . KeyringResponses , len ( dcs ) )
2017-12-05 03:35:32 +00:00
2016-06-20 20:50:59 +00:00
for _ , dc := range dcs {
2014-10-08 20:28:59 +00:00
go func ( dc string ) {
2020-08-11 11:35:48 +00:00
rr := & structs . KeyringResponses { }
2014-10-08 20:28:59 +00:00
if err := s . forwardDC ( method , dc , args , & rr ) ; err != nil {
2014-10-02 06:09:00 +00:00
errorCh <- err
return
}
respCh <- rr
2014-10-08 20:28:59 +00:00
} ( dc )
2014-10-02 06:09:00 +00:00
}
2020-08-11 11:35:48 +00:00
responses := & structs . KeyringResponses { }
for i := 0 ; i < len ( dcs ) ; i ++ {
2014-10-02 06:09:00 +00:00
select {
case err := <- errorCh :
2020-08-11 11:35:48 +00:00
return nil , err
2014-10-02 06:09:00 +00:00
case rr := <- respCh :
2020-08-11 11:35:48 +00:00
responses . Add ( rr )
2014-10-02 06:09:00 +00:00
}
}
2020-08-11 11:35:48 +00:00
return responses , nil
2014-10-02 06:09:00 +00:00
}
2019-09-20 18:37:22 +00:00
type raftEncoder func ( structs . MessageType , interface { } ) ( [ ] byte , error )
2021-04-08 22:58:15 +00:00
// raftApplyMsgpack encodes the msg using msgpack and calls raft.Apply. See
// raftApplyWithEncoder.
// Deprecated: use raftApplyMsgpack
2013-12-19 20:03:57 +00:00
func ( s * Server ) raftApply ( t structs . MessageType , msg interface { } ) ( interface { } , error ) {
2019-09-20 18:37:22 +00:00
return s . raftApplyMsgpack ( t , msg )
}
2021-04-08 22:58:15 +00:00
// raftApplyMsgpack encodes the msg using msgpack and calls raft.Apply. See
// raftApplyWithEncoder.
2019-09-20 18:37:22 +00:00
func ( s * Server ) raftApplyMsgpack ( t structs . MessageType , msg interface { } ) ( interface { } , error ) {
return s . raftApplyWithEncoder ( t , msg , structs . Encode )
}
2021-04-08 22:58:15 +00:00
// raftApplyProtobuf encodes the msg using protobuf and calls raft.Apply. See
// raftApplyWithEncoder.
2019-09-20 18:37:22 +00:00
func ( s * Server ) raftApplyProtobuf ( t structs . MessageType , msg interface { } ) ( interface { } , error ) {
2019-10-24 18:38:09 +00:00
return s . raftApplyWithEncoder ( t , msg , structs . EncodeProtoInterface )
2019-09-20 18:37:22 +00:00
}
2021-04-08 22:58:15 +00:00
// raftApplyWithEncoder encodes a message, and then calls raft.Apply with the
// encoded message. Returns the FSM response along with any errors. If the
// FSM.Apply response is an error it will be returned as the error return
// value with a nil response.
func ( s * Server ) raftApplyWithEncoder (
t structs . MessageType ,
msg interface { } ,
encoder raftEncoder ,
) ( response interface { } , err error ) {
2019-09-20 18:37:22 +00:00
if encoder == nil {
return nil , fmt . Errorf ( "Failed to encode request: nil encoder" )
}
buf , err := encoder ( t , msg )
2013-12-11 22:04:44 +00:00
if err != nil {
return nil , fmt . Errorf ( "Failed to encode request: %v" , err )
}
2014-04-29 05:25:09 +00:00
// Warn if the command is very large
if n := len ( buf ) ; n > raftWarnSize {
2020-01-28 23:50:41 +00:00
s . rpcLogger ( ) . Warn ( "Attempting to apply large raft entry" , "size_in_bytes" , n )
2014-04-29 05:25:09 +00:00
}
2019-07-24 21:06:39 +00:00
var chunked bool
var future raft . ApplyFuture
switch {
case len ( buf ) <= raft . SuggestedMaxDataSize || t != structs . KVSRequestType :
future = s . raft . Apply ( buf , enqueueLimit )
default :
chunked = true
future = raftchunking . ChunkingApply ( buf , nil , enqueueLimit , s . raft . ApplyLog )
}
2013-12-11 22:04:44 +00:00
if err := future . Error ( ) ; err != nil {
return nil , err
}
2019-07-24 21:06:39 +00:00
resp := future . Response ( )
if chunked {
// In this case we didn't apply all chunks successfully, possibly due
// to a term change; resubmit
if resp == nil {
2021-04-08 22:58:15 +00:00
return nil , ErrChunkingResubmit
2019-07-24 21:06:39 +00:00
}
// We expect that this conversion should always work
chunkedSuccess , ok := resp . ( raftchunking . ChunkingSuccess )
if ! ok {
return nil , errors . New ( "unknown type of response back from chunking FSM" )
}
2021-04-08 22:58:15 +00:00
resp = chunkedSuccess . Response
2019-07-24 21:06:39 +00:00
}
2021-04-08 22:58:15 +00:00
if err , ok := resp . ( error ) ; ok {
return nil , err
}
2019-07-24 21:06:39 +00:00
return resp , nil
2013-12-11 22:04:44 +00:00
}
2014-02-05 18:38:29 +00:00
2017-01-13 19:17:38 +00:00
// queryFn is used to perform a query operation. If a re-query is needed, the
2017-01-25 17:45:25 +00:00
// passed-in watch set will be used to block for changes. The passed-in state
// store should be used (vs. calling fsm.State()) since the given state store
// will be correctly watched for changes if the state store is restored from
// a snapshot.
2017-04-21 00:46:29 +00:00
type queryFn func ( memdb . WatchSet , * state . Store ) error
2017-01-13 19:17:38 +00:00
// blockingQuery is used to process a potentially blocking query operation.
2019-09-26 13:55:02 +00:00
func ( s * Server ) blockingQuery ( queryOpts structs . QueryOptionsCompat , queryMeta structs . QueryMetaCompat , fn queryFn ) error {
2020-06-24 16:36:54 +00:00
var cancel func ( )
var ctx context . Context = & lib . StopChannelContext { StopCh : s . shutdownCh }
2020-02-10 18:01:15 +00:00
var queriesBlocking uint64
2019-09-20 18:37:22 +00:00
var queryTimeout time . Duration
2017-01-13 19:17:38 +00:00
2020-02-10 18:01:15 +00:00
// Instrument all queries run
metrics . IncrCounter ( [ ] string { "rpc" , "query" } , 1 )
2019-09-20 18:37:22 +00:00
minQueryIndex := queryOpts . GetMinQueryIndex ( )
2017-01-13 19:17:38 +00:00
// Fast path right to the non-blocking query.
2019-09-20 18:37:22 +00:00
if minQueryIndex == 0 {
2017-01-13 19:17:38 +00:00
goto RUN_QUERY
}
2019-09-20 18:37:22 +00:00
queryTimeout = queryOpts . GetMaxQueryTime ( )
2017-01-13 19:17:38 +00:00
// Restrict the max query time, and ensure there is always one.
2020-01-17 13:20:57 +00:00
if queryTimeout > s . config . MaxQueryTime {
queryTimeout = s . config . MaxQueryTime
2019-09-20 18:37:22 +00:00
} else if queryTimeout <= 0 {
2020-01-17 13:20:57 +00:00
queryTimeout = s . config . DefaultQueryTime
2017-01-13 19:17:38 +00:00
}
// Apply a small amount of jitter to the request.
2021-05-27 21:29:43 +00:00
queryTimeout += lib . RandomStagger ( queryTimeout / structs . JitterFraction )
2017-01-13 19:17:38 +00:00
2020-06-24 16:36:54 +00:00
// wrap the base context with a deadline
ctx , cancel = context . WithDeadline ( ctx , time . Now ( ) . Add ( queryTimeout ) )
defer cancel ( )
2017-01-13 19:17:38 +00:00
2020-02-10 18:01:15 +00:00
// instrument blockingQueries
// atomic inc our server's count of in-flight blockingQueries and store the new value
queriesBlocking = atomic . AddUint64 ( & s . queriesBlocking , 1 )
// atomic dec when we return from blockingQuery()
defer atomic . AddUint64 ( & s . queriesBlocking , ^ uint64 ( 0 ) )
// set the gauge directly to the new value of s.blockingQueries
metrics . SetGauge ( [ ] string { "rpc" , "queries_blocking" } , float32 ( queriesBlocking ) )
2017-01-13 19:17:38 +00:00
RUN_QUERY :
2020-02-10 18:01:15 +00:00
// Setup blocking loop
2017-01-13 19:17:38 +00:00
// Update the query metadata.
s . setQueryMeta ( queryMeta )
2020-02-10 18:01:15 +00:00
// Validate
2017-01-13 19:17:38 +00:00
// If the read must be consistent we verify that we are still the leader.
2019-09-20 18:37:22 +00:00
if queryOpts . GetRequireConsistent ( ) {
2017-01-13 19:17:38 +00:00
if err := s . consistentRead ( ) ; err != nil {
return err
}
}
2020-02-10 18:01:15 +00:00
// Run query
2017-01-19 19:12:29 +00:00
2017-01-25 01:23:48 +00:00
// Operate on a consistent set of state. This makes sure that the
// abandon channel goes with the state that the caller is using to
// build watches.
state := s . fsm . State ( )
2017-01-19 19:12:29 +00:00
// We can skip all watch tracking if this isn't a blocking query.
var ws memdb . WatchSet
2019-09-20 18:37:22 +00:00
if minQueryIndex > 0 {
2017-01-19 19:12:29 +00:00
ws = memdb . NewWatchSet ( )
2017-01-24 18:38:03 +00:00
// This channel will be closed if a snapshot is restored and the
// whole state store is abandoned.
2017-01-25 01:23:48 +00:00
ws . Add ( state . AbandonCh ( ) )
2017-01-19 19:12:29 +00:00
}
2017-01-13 19:17:38 +00:00
2020-02-10 18:01:15 +00:00
// Execute the queryFn
2017-01-25 01:23:48 +00:00
err := fn ( ws , state )
2018-07-25 19:26:27 +00:00
// Note we check queryOpts.MinQueryIndex is greater than zero to determine if
// blocking was requested by client, NOT meta.Index since the state function
2019-03-06 17:13:28 +00:00
// might return zero if something is not initialized and care wasn't taken to
2018-07-25 19:26:27 +00:00
// handle that special case (in practice this happened a lot so fixing it
// systematically here beats trying to remember to add zero checks in every
// state method). We also need to ensure that unless there is an error, we
// return an index > 0 otherwise the client will never block and burn CPU and
// requests.
2019-09-20 18:37:22 +00:00
if err == nil && queryMeta . GetIndex ( ) < 1 {
queryMeta . SetIndex ( 1 )
2018-07-25 19:26:27 +00:00
}
2020-02-10 18:01:15 +00:00
// block up to the timeout if we don't see anything fresh.
2019-09-20 18:37:22 +00:00
if err == nil && minQueryIndex > 0 && queryMeta . GetIndex ( ) <= minQueryIndex {
2020-06-24 16:36:54 +00:00
if err := ws . WatchCtx ( ctx ) ; err == nil {
// a non-nil error only occurs when the context is cancelled
2017-01-26 03:00:32 +00:00
// If a restore may have woken us up then bail out from
// the query immediately. This is slightly race-ey since
// this might have been interrupted for other reasons,
// but it's OK to kick it back to the caller in either
// case.
select {
case <- state . AbandonCh ( ) :
default :
2020-02-10 18:01:15 +00:00
// loop back and look for an update again
2017-01-26 03:00:32 +00:00
goto RUN_QUERY
}
2017-01-13 19:17:38 +00:00
}
}
return err
}
2014-04-19 00:37:19 +00:00
// setQueryMeta is used to populate the QueryMeta data for an RPC call
2019-09-26 13:55:02 +00:00
func ( s * Server ) setQueryMeta ( m structs . QueryMetaCompat ) {
2014-04-19 00:37:19 +00:00
if s . IsLeader ( ) {
2019-09-20 18:37:22 +00:00
m . SetLastContact ( 0 )
m . SetKnownLeader ( true )
2014-04-19 00:37:19 +00:00
} else {
2019-09-20 18:37:22 +00:00
m . SetLastContact ( time . Since ( s . raft . LastContact ( ) ) )
m . SetKnownLeader ( s . raft . Leader ( ) != "" )
2014-04-19 00:37:19 +00:00
}
}
2014-04-19 00:49:01 +00:00
// consistentRead is used to ensure we do not perform a stale
// read. This is done by verifying leadership before the read.
func ( s * Server ) consistentRead ( ) error {
2017-10-04 23:43:27 +00:00
defer metrics . MeasureSince ( [ ] string { "rpc" , "consistentRead" } , time . Now ( ) )
2014-04-19 00:49:01 +00:00
future := s . raft . VerifyLeader ( )
2017-06-16 15:49:54 +00:00
if err := future . Error ( ) ; err != nil {
2017-06-16 03:41:30 +00:00
return err //fail fast if leader verification fails
}
// poll consistent read readiness, wait for up to RPCHoldTimeout milliseconds
if s . isReadyForConsistentReads ( ) {
return nil
}
2021-05-27 21:29:43 +00:00
jitter := lib . RandomStagger ( s . config . RPCHoldTimeout / structs . JitterFraction )
2017-06-16 15:49:54 +00:00
deadline := time . Now ( ) . Add ( s . config . RPCHoldTimeout )
2017-06-16 03:41:30 +00:00
2017-06-16 15:49:54 +00:00
for time . Now ( ) . Before ( deadline ) {
2017-06-16 03:41:30 +00:00
select {
case <- time . After ( jitter ) :
// Drop through and check before we loop again.
case <- s . shutdownCh :
return fmt . Errorf ( "shutdown waiting for leader" )
}
if s . isReadyForConsistentReads ( ) {
return nil
}
}
return structs . ErrNotReadyForConsistentReads
2014-04-19 00:49:01 +00:00
}