2013-12-06 23:43:07 +00:00
|
|
|
package consul
|
|
|
|
|
|
|
|
import (
|
|
|
|
"fmt"
|
|
|
|
"github.com/hashicorp/raft"
|
|
|
|
"github.com/hashicorp/serf/serf"
|
|
|
|
"log"
|
2013-12-07 00:35:13 +00:00
|
|
|
"net"
|
|
|
|
"net/rpc"
|
2013-12-06 23:43:07 +00:00
|
|
|
"os"
|
|
|
|
"path/filepath"
|
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
)
|
|
|
|
|
|
|
|
const (
|
2013-12-07 00:05:26 +00:00
|
|
|
serfLANSnapshot = "serf/local.snapshot"
|
|
|
|
serfWANSnapshot = "serf/remote.snapshot"
|
|
|
|
raftState = "raft/"
|
2013-12-06 23:43:07 +00:00
|
|
|
)
|
|
|
|
|
|
|
|
// Server is Consul server which manages the service discovery,
|
|
|
|
// health checking, DC forwarding, Raft, and multiple Serf pools.
|
|
|
|
type Server struct {
|
|
|
|
config *Config
|
|
|
|
|
2013-12-09 20:09:57 +00:00
|
|
|
// Connection pool to other consul servers
|
|
|
|
connPool *ConnPool
|
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// eventChLAN is used to receive events from the
|
|
|
|
// serf cluster in the datacenter
|
|
|
|
eventChLAN chan serf.Event
|
2013-12-06 23:43:07 +00:00
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// eventChWAN is used to receive events from the
|
|
|
|
// serf cluster that spans datacenters
|
|
|
|
eventChWAN chan serf.Event
|
2013-12-06 23:43:07 +00:00
|
|
|
|
|
|
|
// fsm is the state machine used with Raft to provide
|
|
|
|
// strong consistency.
|
|
|
|
fsm *consulFSM
|
|
|
|
|
|
|
|
// Logger uses the provided LogOutput
|
|
|
|
logger *log.Logger
|
|
|
|
|
|
|
|
// The raft instance is used among Consul nodes within the
|
|
|
|
// DC to protect operations that require strong consistency
|
2013-12-09 22:20:56 +00:00
|
|
|
raft *raft.Raft
|
|
|
|
raftLayer *RaftLayer
|
2013-12-09 23:29:01 +00:00
|
|
|
raftPeers raft.PeerStore
|
2013-12-09 22:20:56 +00:00
|
|
|
raftStore *raft.SQLiteStore
|
|
|
|
raftTransport *raft.NetworkTransport
|
2013-12-06 23:43:07 +00:00
|
|
|
|
2013-12-07 00:35:13 +00:00
|
|
|
// rpcClients is used to track active clients
|
|
|
|
rpcClients map[net.Conn]struct{}
|
|
|
|
rpcClientLock sync.Mutex
|
|
|
|
|
|
|
|
// rpcListener is used to listen for incoming connections
|
|
|
|
rpcListener net.Listener
|
|
|
|
rpcServer *rpc.Server
|
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// serfLAN is the Serf cluster maintained inside the DC
|
2013-12-06 23:43:07 +00:00
|
|
|
// which contains all the DC nodes
|
2013-12-07 00:05:26 +00:00
|
|
|
serfLAN *serf.Serf
|
2013-12-06 23:43:07 +00:00
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// serfWAN is the Serf cluster maintained between DC's
|
2013-12-06 23:43:07 +00:00
|
|
|
// which SHOULD only consist of Consul servers
|
2013-12-07 00:05:26 +00:00
|
|
|
serfWAN *serf.Serf
|
2013-12-06 23:43:07 +00:00
|
|
|
|
|
|
|
shutdown bool
|
|
|
|
shutdownCh chan struct{}
|
|
|
|
shutdownLock sync.Mutex
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewServer is used to construct a new Consul server from the
|
|
|
|
// configuration, potentially returning an error
|
|
|
|
func NewServer(config *Config) (*Server, error) {
|
|
|
|
// Check for a data directory!
|
|
|
|
if config.DataDir == "" {
|
|
|
|
return nil, fmt.Errorf("Config must provide a DataDir")
|
|
|
|
}
|
|
|
|
|
|
|
|
// Ensure we have a log output
|
|
|
|
if config.LogOutput == nil {
|
|
|
|
config.LogOutput = os.Stderr
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create a logger
|
|
|
|
logger := log.New(config.LogOutput, "", log.LstdFlags)
|
|
|
|
|
|
|
|
// Create server
|
|
|
|
s := &Server{
|
2013-12-07 00:05:26 +00:00
|
|
|
config: config,
|
2013-12-09 21:13:40 +00:00
|
|
|
connPool: NewPool(5),
|
2013-12-07 00:05:26 +00:00
|
|
|
eventChLAN: make(chan serf.Event, 256),
|
|
|
|
eventChWAN: make(chan serf.Event, 256),
|
|
|
|
logger: logger,
|
2013-12-07 00:35:13 +00:00
|
|
|
rpcClients: make(map[net.Conn]struct{}),
|
|
|
|
rpcServer: rpc.NewServer(),
|
2013-12-07 00:05:26 +00:00
|
|
|
shutdownCh: make(chan struct{}),
|
2013-12-06 23:43:07 +00:00
|
|
|
}
|
|
|
|
|
2013-12-09 21:13:40 +00:00
|
|
|
// Initialize the RPC layer
|
|
|
|
if err := s.setupRPC(); err != nil {
|
|
|
|
s.Shutdown()
|
|
|
|
return nil, fmt.Errorf("Failed to start RPC layer: %v", err)
|
|
|
|
}
|
|
|
|
|
2013-12-09 23:29:01 +00:00
|
|
|
// Initialize the Raft server
|
|
|
|
if err := s.setupRaft(); err != nil {
|
|
|
|
s.Shutdown()
|
|
|
|
return nil, fmt.Errorf("Failed to start Raft: %v", err)
|
|
|
|
}
|
|
|
|
|
2013-12-06 23:43:07 +00:00
|
|
|
// Start the Serf listeners to prevent a deadlock
|
2013-12-07 00:05:26 +00:00
|
|
|
go s.lanEventHandler()
|
|
|
|
go s.wanEventHandler()
|
2013-12-06 23:43:07 +00:00
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// Initialize the lan Serf
|
2013-12-06 23:43:07 +00:00
|
|
|
var err error
|
2013-12-07 01:18:09 +00:00
|
|
|
s.serfLAN, err = s.setupSerf(config.SerfLANConfig,
|
2013-12-07 00:05:26 +00:00
|
|
|
s.eventChLAN, serfLANSnapshot)
|
2013-12-06 23:43:07 +00:00
|
|
|
if err != nil {
|
|
|
|
s.Shutdown()
|
2013-12-07 00:05:26 +00:00
|
|
|
return nil, fmt.Errorf("Failed to start lan serf: %v", err)
|
2013-12-06 23:43:07 +00:00
|
|
|
}
|
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
// Initialize the wan Serf
|
2013-12-07 01:18:09 +00:00
|
|
|
s.serfWAN, err = s.setupSerf(config.SerfWANConfig,
|
2013-12-07 00:05:26 +00:00
|
|
|
s.eventChWAN, serfWANSnapshot)
|
2013-12-06 23:43:07 +00:00
|
|
|
if err != nil {
|
|
|
|
s.Shutdown()
|
2013-12-07 00:05:26 +00:00
|
|
|
return nil, fmt.Errorf("Failed to start wan serf: %v", err)
|
2013-12-06 23:43:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return s, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// ensurePath is used to make sure a path exists
|
|
|
|
func (s *Server) ensurePath(path string, dir bool) error {
|
|
|
|
if !dir {
|
|
|
|
path = filepath.Dir(path)
|
|
|
|
}
|
|
|
|
return os.MkdirAll(path, 0755)
|
|
|
|
}
|
|
|
|
|
|
|
|
// setupSerf is used to setup and initialize a Serf
|
|
|
|
func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string) (*serf.Serf, error) {
|
2013-12-10 00:05:15 +00:00
|
|
|
addr := s.rpcListener.Addr().(*net.TCPAddr)
|
2013-12-06 23:43:07 +00:00
|
|
|
conf.NodeName = s.config.NodeName
|
2013-12-10 00:05:15 +00:00
|
|
|
conf.Role = fmt.Sprintf("consul:%s:%d", s.config.Datacenter, addr.Port)
|
2013-12-06 23:43:07 +00:00
|
|
|
conf.MemberlistConfig.LogOutput = s.config.LogOutput
|
|
|
|
conf.LogOutput = s.config.LogOutput
|
|
|
|
conf.EventCh = ch
|
|
|
|
conf.SnapshotPath = filepath.Join(s.config.DataDir, path)
|
|
|
|
if err := s.ensurePath(conf.SnapshotPath, false); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
return serf.Create(conf)
|
|
|
|
}
|
|
|
|
|
|
|
|
// setupRaft is used to setup and initialize Raft
|
|
|
|
func (s *Server) setupRaft() error {
|
|
|
|
// Create the base path
|
|
|
|
path := filepath.Join(s.config.DataDir, raftState)
|
|
|
|
if err := s.ensurePath(path, true); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create the SQLite store for logs and stable storage
|
|
|
|
store, err := raft.NewSQLiteStore(path)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
2013-12-09 21:13:40 +00:00
|
|
|
s.raftStore = store
|
2013-12-06 23:43:07 +00:00
|
|
|
|
|
|
|
// Create the snapshot store
|
|
|
|
snapshots, err := raft.NewFileSnapshotStore(path, 3)
|
|
|
|
if err != nil {
|
|
|
|
store.Close()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Create a transport layer
|
2013-12-09 21:13:40 +00:00
|
|
|
trans := raft.NewNetworkTransport(s.raftLayer, 3, 10*time.Second)
|
2013-12-09 22:20:56 +00:00
|
|
|
s.raftTransport = trans
|
2013-12-06 23:43:07 +00:00
|
|
|
|
|
|
|
// Setup the peer store
|
2013-12-09 23:29:01 +00:00
|
|
|
s.raftPeers = raft.NewJSONPeers(path, trans)
|
2013-12-06 23:43:07 +00:00
|
|
|
|
|
|
|
// Create the FSM
|
|
|
|
s.fsm = &consulFSM{server: s}
|
|
|
|
|
|
|
|
// Setup the Raft store
|
2013-12-09 23:29:01 +00:00
|
|
|
s.raft, err = raft.NewRaft(s.config.RaftConfig, s.fsm, store, store,
|
|
|
|
snapshots, s.raftPeers, trans)
|
2013-12-06 23:43:07 +00:00
|
|
|
if err != nil {
|
|
|
|
store.Close()
|
|
|
|
trans.Close()
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2013-12-07 00:35:13 +00:00
|
|
|
// setupRPC is used to setup the RPC listener
|
|
|
|
func (s *Server) setupRPC() error {
|
2013-12-09 22:49:07 +00:00
|
|
|
// Register the handlers
|
2013-12-09 23:29:01 +00:00
|
|
|
s.rpcServer.Register(&Status{server: s})
|
2013-12-09 22:49:07 +00:00
|
|
|
s.rpcServer.Register(&Raft{server: s})
|
|
|
|
|
2013-12-07 00:35:13 +00:00
|
|
|
list, err := net.Listen("tcp", s.config.RPCAddr)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
s.rpcListener = list
|
2013-12-09 22:25:59 +00:00
|
|
|
s.raftLayer = NewRaftLayer(s.rpcListener.Addr())
|
2013-12-07 00:35:13 +00:00
|
|
|
go s.listen()
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2013-12-06 23:43:07 +00:00
|
|
|
// Shutdown is used to shutdown the server
|
|
|
|
func (s *Server) Shutdown() error {
|
|
|
|
s.logger.Printf("[INFO] Shutting down Consul server")
|
|
|
|
s.shutdownLock.Lock()
|
|
|
|
defer s.shutdownLock.Unlock()
|
|
|
|
|
|
|
|
if s.shutdown {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
s.shutdown = true
|
|
|
|
close(s.shutdownCh)
|
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
if s.serfLAN != nil {
|
|
|
|
s.serfLAN.Shutdown()
|
2013-12-06 23:43:07 +00:00
|
|
|
}
|
|
|
|
|
2013-12-07 00:05:26 +00:00
|
|
|
if s.serfWAN != nil {
|
|
|
|
s.serfWAN.Shutdown()
|
2013-12-06 23:43:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if s.raft != nil {
|
2013-12-09 22:20:56 +00:00
|
|
|
s.raftTransport.Close()
|
2013-12-09 21:13:40 +00:00
|
|
|
s.raftLayer.Close()
|
2013-12-09 22:20:56 +00:00
|
|
|
s.raft.Shutdown()
|
2013-12-06 23:43:07 +00:00
|
|
|
s.raftStore.Close()
|
|
|
|
}
|
2013-12-07 00:35:13 +00:00
|
|
|
|
|
|
|
if s.rpcListener != nil {
|
|
|
|
s.rpcListener.Close()
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close all the RPC connections
|
|
|
|
s.rpcClientLock.Lock()
|
|
|
|
for conn := range s.rpcClients {
|
|
|
|
conn.Close()
|
|
|
|
}
|
|
|
|
s.rpcClientLock.Unlock()
|
|
|
|
|
2013-12-09 20:09:57 +00:00
|
|
|
// Close the connection pool
|
|
|
|
s.connPool.Shutdown()
|
|
|
|
|
2013-12-06 23:43:07 +00:00
|
|
|
return nil
|
|
|
|
}
|
2013-12-07 01:18:09 +00:00
|
|
|
|
2013-12-09 20:10:27 +00:00
|
|
|
// Leave is used to prepare for a graceful shutdown of the server
|
|
|
|
func (s *Server) Leave() error {
|
|
|
|
s.logger.Printf("[INFO] Consul server starting leave")
|
|
|
|
|
|
|
|
// Leave the WAN pool
|
|
|
|
if s.serfWAN != nil {
|
|
|
|
if err := s.serfWAN.Leave(); err != nil {
|
|
|
|
s.logger.Printf("[ERR] Failed to leave WAN Serf cluster: %v", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Leave the LAN pool
|
|
|
|
if s.serfLAN != nil {
|
|
|
|
if err := s.serfLAN.Leave(); err != nil {
|
|
|
|
s.logger.Printf("[ERR] Failed to leave LAN Serf cluster: %v", err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Leave the Raft cluster
|
|
|
|
if s.raft != nil {
|
|
|
|
// Get the leader
|
|
|
|
leader := s.raft.Leader()
|
|
|
|
if leader == nil {
|
|
|
|
s.logger.Printf("[ERR] Failed to leave Raft cluster: no leader")
|
|
|
|
goto AFTER_LEAVE
|
|
|
|
}
|
|
|
|
|
|
|
|
// Request that we are removed
|
|
|
|
ch := make(chan error, 1)
|
|
|
|
go func() {
|
2013-12-09 23:01:42 +00:00
|
|
|
var out struct{}
|
|
|
|
peer := s.rpcListener.Addr().String()
|
|
|
|
err := s.connPool.RPC(leader, "Raft.RemovePeer", peer, &out)
|
|
|
|
ch <- err
|
2013-12-09 20:10:27 +00:00
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for the commit
|
|
|
|
select {
|
|
|
|
case err := <-ch:
|
|
|
|
if err != nil {
|
|
|
|
s.logger.Printf("[ERR] Failed to leave Raft cluster: %v", err)
|
|
|
|
}
|
|
|
|
case <-time.After(3 * time.Second):
|
|
|
|
s.logger.Printf("[ERR] Timedout leaving Raft cluster")
|
|
|
|
}
|
|
|
|
}
|
|
|
|
AFTER_LEAVE:
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2013-12-07 01:18:09 +00:00
|
|
|
// JoinLAN is used to have Consul join the inner-DC pool
|
|
|
|
// The target address should be another node inside the DC
|
|
|
|
// listening on the Serf LAN address
|
|
|
|
func (s *Server) JoinLAN(addr string) error {
|
|
|
|
_, err := s.serfLAN.Join([]string{addr}, false)
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// JoinWAN is used to have Consul join the cross-WAN Consul ring
|
|
|
|
// The target address should be another node listening on the
|
|
|
|
// Serf WAN address
|
|
|
|
func (s *Server) JoinWAN(addr string) error {
|
|
|
|
_, err := s.serfWAN.Join([]string{addr}, false)
|
|
|
|
return err
|
|
|
|
}
|
2013-12-10 21:17:54 +00:00
|
|
|
|
|
|
|
// LANMembers is used to return the members of the LAN cluster
|
|
|
|
func (s *Server) LANMembers() []serf.Member {
|
|
|
|
return s.serfLAN.Members()
|
|
|
|
}
|
|
|
|
|
|
|
|
// WANMembers is used to return the members of the LAN cluster
|
|
|
|
func (s *Server) WANMembers() []serf.Member {
|
|
|
|
return s.serfWAN.Members()
|
|
|
|
}
|