2013-12-07 00:54:33 +00:00
|
|
|
package consul
|
|
|
|
|
|
|
|
import (
|
2013-12-11 22:04:44 +00:00
|
|
|
"fmt"
|
2013-12-19 20:03:57 +00:00
|
|
|
"github.com/hashicorp/consul/consul/structs"
|
2013-12-07 00:54:33 +00:00
|
|
|
"github.com/ugorji/go/codec"
|
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"
|
2014-02-05 18:38:29 +00:00
|
|
|
"time"
|
2013-12-07 00:54:33 +00:00
|
|
|
)
|
|
|
|
|
2013-12-09 21:13:40 +00:00
|
|
|
type RPCType byte
|
|
|
|
|
|
|
|
const (
|
|
|
|
rpcConsul RPCType = iota
|
|
|
|
rpcRaft
|
|
|
|
)
|
|
|
|
|
2014-02-05 18:38:29 +00:00
|
|
|
const (
|
|
|
|
maxQueryTime = 600 * 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
|
|
|
|
}
|
|
|
|
|
|
|
|
// Track this client
|
|
|
|
s.rpcClientLock.Lock()
|
|
|
|
s.rpcClients[conn] = struct{}{}
|
|
|
|
s.rpcClientLock.Unlock()
|
|
|
|
|
|
|
|
go s.handleConn(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
|
2013-12-07 00:54:33 +00:00
|
|
|
func (s *Server) handleConn(conn net.Conn) {
|
2013-12-09 21:13:40 +00:00
|
|
|
// Read a single byte
|
|
|
|
buf := make([]byte, 1)
|
|
|
|
if _, err := conn.Read(buf); err != nil {
|
2014-01-10 19:06:11 +00:00
|
|
|
s.logger.Printf("[ERR] consul.rpc: failed to read byte: %v", err)
|
2013-12-09 21:13:40 +00:00
|
|
|
conn.Close()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
// Switch on the byte
|
|
|
|
switch RPCType(buf[0]) {
|
|
|
|
case rpcConsul:
|
|
|
|
s.handleConsulConn(conn)
|
|
|
|
|
|
|
|
case rpcRaft:
|
|
|
|
s.raftLayer.Handoff(conn)
|
|
|
|
|
|
|
|
default:
|
2014-01-10 19:06:11 +00:00
|
|
|
s.logger.Printf("[ERR] consul.rpc: unrecognized RPC byte: %v", buf[0])
|
2013-12-09 21:13:40 +00:00
|
|
|
conn.Close()
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// handleConsulConn is used to service a single Consul RPC connection
|
|
|
|
func (s *Server) handleConsulConn(conn net.Conn) {
|
2013-12-07 00:54:33 +00:00
|
|
|
defer func() {
|
|
|
|
conn.Close()
|
|
|
|
s.rpcClientLock.Lock()
|
|
|
|
delete(s.rpcClients, conn)
|
|
|
|
s.rpcClientLock.Unlock()
|
|
|
|
}()
|
|
|
|
|
|
|
|
rpcCodec := codec.GoRpc.ServerCodec(conn, &codec.MsgpackHandle{})
|
|
|
|
for !s.shutdown {
|
|
|
|
if err := s.rpcServer.ServeRequest(rpcCodec); err != nil {
|
2013-12-11 22:57:40 +00:00
|
|
|
if err != io.EOF {
|
2014-01-10 19:06:11 +00:00
|
|
|
s.logger.Printf("[ERR] consul.rpc: RPC error: %v (%v)", err, conn)
|
2013-12-11 22:57:40 +00:00
|
|
|
}
|
2013-12-07 00:54:33 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2013-12-11 22:04:44 +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
|
|
|
|
func (s *Server) forward(method, dc string, args interface{}, reply interface{}) (bool, error) {
|
|
|
|
// Handle DC forwarding
|
|
|
|
if dc != s.config.Datacenter {
|
|
|
|
err := s.forwardDC(method, dc, args, reply)
|
|
|
|
return true, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// 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 {
|
|
|
|
leader := s.raft.Leader()
|
|
|
|
if leader == nil {
|
2013-12-19 20:03:57 +00:00
|
|
|
return structs.ErrNoLeader
|
2013-12-11 22:04:44 +00:00
|
|
|
}
|
|
|
|
return s.connPool.RPC(leader, 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
|
|
|
|
return s.connPool.RPC(server, method, args, reply)
|
2013-12-11 22:04:44 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// 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)
|
2013-12-11 22:04:44 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, fmt.Errorf("Failed to encode request: %v", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
future := s.raft.Apply(buf, 0)
|
|
|
|
if err := future.Error(); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
return future.Response(), nil
|
|
|
|
}
|
2014-02-05 18:38:29 +00:00
|
|
|
|
|
|
|
// 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.BlockingQuery, tables MDBTables, run func() (uint64, error)) error {
|
|
|
|
var timeout <-chan time.Time
|
|
|
|
var notifyCh chan struct{}
|
|
|
|
|
|
|
|
// Fast path non-blocking
|
|
|
|
if b.MinQueryIndex == 0 {
|
|
|
|
goto RUN_QUERY
|
|
|
|
}
|
|
|
|
|
2014-02-05 19:13:08 +00:00
|
|
|
// Sanity check that we have tables to block on
|
|
|
|
if len(tables) == 0 {
|
|
|
|
panic("no tables to block on")
|
|
|
|
}
|
|
|
|
|
2014-02-05 18:38:29 +00:00
|
|
|
// Restrict the max query time
|
|
|
|
if b.MaxQueryTime > maxQueryTime {
|
|
|
|
b.MaxQueryTime = maxQueryTime
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure a time limit is set if we have an index
|
|
|
|
if b.MinQueryIndex > 0 && b.MaxQueryTime == 0 {
|
|
|
|
b.MaxQueryTime = maxQueryTime
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup a query timeout
|
|
|
|
if b.MaxQueryTime > 0 {
|
|
|
|
timeout = time.After(b.MaxQueryTime)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup a notification channel for changes
|
|
|
|
SETUP_NOTIFY:
|
|
|
|
if b.MinQueryIndex > 0 {
|
|
|
|
notifyCh = make(chan struct{}, 1)
|
|
|
|
s.fsm.State().Watch(tables, notifyCh)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Run the query function
|
|
|
|
RUN_QUERY:
|
|
|
|
idx, err := run()
|
|
|
|
|
|
|
|
// Check for minimum query time
|
|
|
|
if err == nil && idx <= b.MinQueryIndex {
|
|
|
|
select {
|
|
|
|
case <-notifyCh:
|
|
|
|
goto SETUP_NOTIFY
|
|
|
|
case <-timeout:
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return err
|
|
|
|
}
|