open-consul/consul/rpc.go

418 lines
11 KiB
Go
Raw Normal View History

2013-12-07 00:54:33 +00:00
package consul
import (
"crypto/tls"
"fmt"
2013-12-11 22:57:40 +00:00
"io"
2013-12-12 00:33:19 +00:00
"math/rand"
2013-12-07 00:54:33 +00:00
"net"
"strings"
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/consul/structs"
"github.com/hashicorp/go-msgpack/codec"
"github.com/hashicorp/yamux"
"github.com/inconshreveable/muxado"
2013-12-07 00:54:33 +00:00
)
type RPCType byte
const (
rpcConsul RPCType = iota
rpcRaft
rpcMultiplex
rpcTLS
rpcMultiplexV2
)
const (
// maxQueryTime is used to bound the limit of a blocking query
maxQueryTime = 600 * time.Second
// defaultQueryTime is the amount of time we block waiting for a change
// if no time is specified. Previously we would wait the maxQueryTime.
defaultQueryTime = 300 * time.Second
// jitterFraction is a the limit to the amount of jitter we apply
// to a user specified MaxQueryTime. We divide the specified time by
// the fraction. So 16 == 6.25% limit of jitter
jitterFraction = 16
// Warn if the Raft command is larger than this.
// If it's over 1MB something is probably being abusive.
raftWarnSize = 1024 * 1024
// 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
)
2013-12-07 00:54:33 +00:00
// listen is used to listen for incoming RPC connections
func (s *Server) listen() {
for {
// Accept a connection
conn, err := s.rpcListener.Accept()
if err != nil {
if s.shutdown {
return
}
2014-01-10 19:06:11 +00:00
s.logger.Printf("[ERR] consul.rpc: failed to accept RPC conn: %v", err)
2013-12-07 00:54:33 +00:00
continue
}
go s.handleConn(conn, false)
2014-02-20 23:16:26 +00:00
metrics.IncrCounter([]string{"consul", "rpc", "accept_conn"}, 1)
2013-12-07 00:54:33 +00:00
}
}
// handleConn is used to determine if this is a Raft or
// Consul type RPC connection and invoke the correct handler
func (s *Server) handleConn(conn net.Conn, isTLS bool) {
// Read a single byte
buf := make([]byte, 1)
if _, err := conn.Read(buf); err != nil {
if err != io.EOF {
s.logger.Printf("[ERR] consul.rpc: failed to read byte: %v", err)
}
conn.Close()
return
}
// Enforce TLS if VerifyIncoming is set
if s.config.VerifyIncoming && !isTLS && RPCType(buf[0]) != rpcTLS {
s.logger.Printf("[WARN] consul.rpc: Non-TLS connection attempted with VerifyIncoming set")
conn.Close()
return
}
// Switch on the byte
switch RPCType(buf[0]) {
case rpcConsul:
s.handleConsulConn(conn)
case rpcRaft:
2014-02-20 23:16:26 +00:00
metrics.IncrCounter([]string{"consul", "rpc", "raft_handoff"}, 1)
s.raftLayer.Handoff(conn)
case rpcMultiplex:
s.handleMultiplex(conn)
case rpcTLS:
if s.rpcTLS == nil {
s.logger.Printf("[WARN] consul.rpc: TLS connection attempted, server not configured for TLS")
conn.Close()
return
}
conn = tls.Server(conn, s.rpcTLS)
s.handleConn(conn, true)
case rpcMultiplexV2:
s.handleMultiplexV2(conn)
default:
2014-01-10 19:06:11 +00:00
s.logger.Printf("[ERR] consul.rpc: unrecognized RPC byte: %v", buf[0])
conn.Close()
return
}
}
// handleMultiplex is used to multiplex a single incoming connection
// using the Muxado multiplexer
func (s *Server) handleMultiplex(conn net.Conn) {
defer conn.Close()
server := muxado.Server(conn)
for {
sub, err := server.Accept()
if err != nil {
if !strings.Contains(err.Error(), "closed") {
s.logger.Printf("[ERR] consul.rpc: multiplex conn accept failed: %v", err)
}
return
}
go s.handleConsulConn(sub)
}
}
// 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()
conf.LogOutput = s.config.LogOutput
server, _ := yamux.Server(conn, conf)
for {
sub, err := server.Accept()
if err != nil {
if err != io.EOF {
s.logger.Printf("[ERR] consul.rpc: multiplex conn accept failed: %v", err)
}
return
}
go s.handleConsulConn(sub)
}
}
// handleConsulConn is used to service a single Consul RPC connection
func (s *Server) handleConsulConn(conn net.Conn) {
defer conn.Close()
2014-06-08 21:02:42 +00:00
rpcCodec := codec.GoRpc.ServerCodec(conn, msgpackHandle)
for {
select {
case <-s.shutdownCh:
return
default:
}
2013-12-07 00:54:33 +00:00
if err := s.rpcServer.ServeRequest(rpcCodec); err != nil {
if err != io.EOF && !strings.Contains(err.Error(), "closed") {
2014-01-10 19:06:11 +00:00
s.logger.Printf("[ERR] consul.rpc: RPC error: %v (%v)", err, conn)
metrics.IncrCounter([]string{"consul", "rpc", "request_error"}, 1)
2013-12-11 22:57:40 +00:00
}
2013-12-07 00:54:33 +00:00
return
}
metrics.IncrCounter([]string{"consul", "rpc", "request"}, 1)
2013-12-07 00:54:33 +00:00
}
}
// forward is used to forward to a remote DC or to forward to the local leader
// Returns a bool of if forwarding was performed, as well as any error
2014-04-19 00:17:12 +00:00
func (s *Server) forward(method string, info structs.RPCInfo, args interface{}, reply interface{}) (bool, error) {
// Handle DC forwarding
2014-04-19 00:17:12 +00:00
dc := info.RequestDatacenter()
if dc != s.config.Datacenter {
err := s.forwardDC(method, dc, args, reply)
return true, err
}
2014-04-19 00:26:59 +00:00
// Check if we can allow a stale read
if info.IsRead() && info.AllowStaleRead() {
return false, nil
}
// Handle leader forwarding
if !s.IsLeader() {
err := s.forwardLeader(method, args, reply)
return true, err
}
return false, nil
}
// forwardLeader is used to forward an RPC call to the leader, or fail if no leader
func (s *Server) forwardLeader(method string, args interface{}, reply interface{}) error {
// Get the leader
leader := s.raft.Leader()
2015-05-08 18:35:12 +00:00
if leader == "" {
2013-12-19 20:03:57 +00:00
return structs.ErrNoLeader
}
// Lookup the server
s.localLock.RLock()
2015-05-08 18:35:12 +00:00
server := s.localConsuls[leader]
s.localLock.RUnlock()
// Handle a missing server
if server == nil {
return structs.ErrNoLeader
}
return s.connPool.RPC(s.config.Datacenter, server.Addr, server.Version, method, args, reply)
}
// 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 {
2013-12-12 00:33:19 +00:00
// Bail if we can't find any servers
s.remoteLock.RLock()
servers := s.remoteConsuls[dc]
if len(servers) == 0 {
s.remoteLock.RUnlock()
2014-01-10 19:06:11 +00:00
s.logger.Printf("[WARN] consul.rpc: RPC request for DC '%s', no path found", dc)
2013-12-19 20:03:57 +00:00
return structs.ErrNoDCPath
2013-12-12 00:33:19 +00:00
}
// Select a random addr
offset := rand.Int31() % int32(len(servers))
server := servers[offset]
s.remoteLock.RUnlock()
// Forward to remote Consul
2014-02-20 23:16:26 +00:00
metrics.IncrCounter([]string{"consul", "rpc", "cross-dc", dc}, 1)
return s.connPool.RPC(dc, server.Addr, server.Version, method, args, reply)
}
// globalRPC is used to forward an RPC request to one server in each datacenter.
// This will only error for RPC-related errors. Otherwise, application-level
2014-10-05 20:15:59 +00:00
// errors can be sent in the response objects.
func (s *Server) globalRPC(method string, args interface{},
reply structs.CompoundResponse) error {
errorCh := make(chan error)
respCh := make(chan interface{})
// Make a new request into each datacenter
for dc, _ := range s.remoteConsuls {
go func(dc string) {
rr := reply.New()
if err := s.forwardDC(method, dc, args, &rr); err != nil {
errorCh <- err
return
}
respCh <- rr
}(dc)
}
replies, total := 0, len(s.remoteConsuls)
for replies < total {
select {
case err := <-errorCh:
return err
case rr := <-respCh:
reply.Add(rr)
2014-10-06 22:14:30 +00:00
replies++
}
}
return nil
}
// raftApply is used to encode a message, run it through raft, and return
// the FSM response along with any errors
2013-12-19 20:03:57 +00:00
func (s *Server) raftApply(t structs.MessageType, msg interface{}) (interface{}, error) {
buf, err := structs.Encode(t, msg)
if err != nil {
return nil, fmt.Errorf("Failed to encode request: %v", err)
}
// Warn if the command is very large
if n := len(buf); n > raftWarnSize {
s.logger.Printf("[WARN] consul: Attempting to apply large raft entry (%d bytes)", n)
}
future := s.raft.Apply(buf, enqueueLimit)
if err := future.Error(); err != nil {
return nil, err
}
return future.Response(), nil
}
// blockingRPC is used for queries that need to wait for a
// minimum index. This is used to block and wait for changes.
func (s *Server) blockingRPC(b *structs.QueryOptions, m *structs.QueryMeta,
tables MDBTables, run func() error) error {
2015-01-06 00:41:57 +00:00
opts := blockingRPCOptions{
queryOpts: b,
queryMeta: m,
tables: tables,
run: run,
}
return s.blockingRPCOpt(&opts)
}
// blockingRPCOptions is used to parameterize blockingRPCOpt since
2015-09-11 19:24:54 +00:00
// it takes so many options. It should be preferred over blockingRPC.
2015-01-06 00:41:57 +00:00
type blockingRPCOptions struct {
queryOpts *structs.QueryOptions
queryMeta *structs.QueryMeta
tables MDBTables
kvWatch bool
kvPrefix string
run func() error
}
// blockingRPCOpt is the replacement for blockingRPC as it allows
2015-09-11 19:24:54 +00:00
// for more parameterization easily. It should be preferred over blockingRPC.
2015-01-06 00:41:57 +00:00
func (s *Server) blockingRPCOpt(opts *blockingRPCOptions) error {
2015-05-15 00:42:47 +00:00
var timeout *time.Timer
var notifyCh chan struct{}
var state *StateStore
// Fast path non-blocking
2015-01-06 00:41:57 +00:00
if opts.queryOpts.MinQueryIndex == 0 {
goto RUN_QUERY
}
2014-02-05 19:13:08 +00:00
// Sanity check that we have tables to block on
2015-01-06 00:41:57 +00:00
if len(opts.tables) == 0 && !opts.kvWatch {
2014-02-05 19:13:08 +00:00
panic("no tables to block on")
}
// Restrict the max query time, and ensure there is always one
2015-01-06 00:41:57 +00:00
if opts.queryOpts.MaxQueryTime > maxQueryTime {
opts.queryOpts.MaxQueryTime = maxQueryTime
} else if opts.queryOpts.MaxQueryTime <= 0 {
opts.queryOpts.MaxQueryTime = defaultQueryTime
}
// Apply a small amount of jitter to the request
opts.queryOpts.MaxQueryTime += randomStagger(opts.queryOpts.MaxQueryTime / jitterFraction)
// Setup a query timeout
2015-05-15 00:42:47 +00:00
timeout = time.NewTimer(opts.queryOpts.MaxQueryTime)
// Setup the notify channel
notifyCh = make(chan struct{}, 1)
// Ensure we tear down any watchers on return
state = s.fsm.State()
defer func() {
2015-05-15 00:42:47 +00:00
timeout.Stop()
state.StopWatch(opts.tables, notifyCh)
2015-01-06 00:41:57 +00:00
if opts.kvWatch {
state.StopWatchKV(opts.kvPrefix, notifyCh)
2015-01-06 00:41:57 +00:00
}
}()
REGISTER_NOTIFY:
// Register the notification channel. This may be done
// multiple times if we have not reached the target wait index.
state.Watch(opts.tables, notifyCh)
if opts.kvWatch {
state.WatchKV(opts.kvPrefix, notifyCh)
}
RUN_QUERY:
// Update the query meta data
2015-01-06 00:41:57 +00:00
s.setQueryMeta(opts.queryMeta)
// Check if query must be consistent
2015-01-06 00:41:57 +00:00
if opts.queryOpts.RequireConsistent {
if err := s.consistentRead(); err != nil {
return err
}
}
// Run the query function
metrics.IncrCounter([]string{"consul", "rpc", "query"}, 1)
2015-01-06 00:41:57 +00:00
err := opts.run()
// Check for minimum query time
2015-01-06 00:41:57 +00:00
if err == nil && opts.queryMeta.Index > 0 && opts.queryMeta.Index <= opts.queryOpts.MinQueryIndex {
select {
case <-notifyCh:
goto REGISTER_NOTIFY
2015-05-15 00:42:47 +00:00
case <-timeout.C:
}
}
return err
}
// setQueryMeta is used to populate the QueryMeta data for an RPC call
func (s *Server) setQueryMeta(m *structs.QueryMeta) {
if s.IsLeader() {
m.LastContact = 0
m.KnownLeader = true
} else {
m.LastContact = time.Now().Sub(s.raft.LastContact())
2015-05-08 18:35:12 +00:00
m.KnownLeader = (s.raft.Leader() != "")
}
}
// 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 {
defer metrics.MeasureSince([]string{"consul", "rpc", "consistentRead"}, time.Now())
future := s.raft.VerifyLeader()
return future.Error()
}