Add a path for transitioning to TLS on an existing cluster (#3001)

Fixes #1705
This commit is contained in:
Kyle Havlovitz 2017-05-10 14:25:48 -07:00 committed by GitHub
parent 6eba69fbfe
commit 5bab68b9bb
16 changed files with 224 additions and 43 deletions

View File

@ -453,6 +453,9 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
// Copy the TLS configuration
base.VerifyIncoming = a.config.VerifyIncoming || a.config.VerifyIncomingRPC
if a.config.CAPath != "" || a.config.CAFile != "" {
base.UseTLS = true
}
base.VerifyOutgoing = a.config.VerifyOutgoing
base.VerifyServerHostname = a.config.VerifyServerHostname
base.CAFile = a.config.CAFile

View File

@ -40,6 +40,9 @@ type Server struct {
NonVoter bool
Addr net.Addr
Status serf.MemberStatus
// If true, use TLS when connecting to this server
UseTLS bool
}
// Key returns the corresponding Key
@ -72,6 +75,8 @@ func IsConsulServer(m serf.Member) (bool, *Server) {
datacenter := m.Tags["dc"]
_, bootstrap := m.Tags["bootstrap"]
_, useTLS := m.Tags["use_tls"]
expect := 0
expect_str, ok := m.Tags["expect"]
var err error
@ -135,6 +140,7 @@ func IsConsulServer(m serf.Member) (bool, *Server) {
RaftVersion: raft_vsn,
Status: m.Status,
NonVoter: nonVoter,
UseTLS: useTLS,
}
return true, parts
}

View File

@ -64,6 +64,7 @@ func TestIsConsulServer(t *testing.T) {
"vsn": "1",
"expect": "3",
"raft_vsn": "3",
"use_tls": "1",
},
Status: serf.StatusLeft,
}
@ -95,6 +96,9 @@ func TestIsConsulServer(t *testing.T) {
if parts.Status != serf.StatusLeft {
t.Fatalf("bad: %v", parts.Status)
}
if !parts.UseTLS {
t.Fatalf("bad: %v", parts.UseTLS)
}
m.Tags["bootstrap"] = "1"
m.Tags["disabled"] = "1"
ok, parts = agent.IsConsulServer(m)

View File

@ -115,7 +115,7 @@ func NewClient(config *Config) (*Client, error) {
// Create server
c := &Client{
config: config,
connPool: NewPool(config.RPCSrcAddr, config.LogOutput, clientRPCConnMaxIdle, clientMaxStreams, tlsWrap),
connPool: NewPool(config.RPCSrcAddr, config.LogOutput, clientRPCConnMaxIdle, clientMaxStreams, tlsWrap, config.VerifyOutgoing),
eventCh: make(chan serf.Event, serfEventBacklog),
logger: logger,
shutdownCh: make(chan struct{}),
@ -334,7 +334,7 @@ func (c *Client) RPC(method string, args interface{}, reply interface{}) error {
}
// Forward to remote Consul
if err := c.connPool.RPC(c.config.Datacenter, server.Addr, server.Version, method, args, reply); err != nil {
if err := c.connPool.RPC(c.config.Datacenter, server.Addr, server.Version, method, server.UseTLS, args, reply); err != nil {
c.servers.NotifyFailedServer(server)
c.logger.Printf("[ERR] consul: RPC failed to server %s: %v", server.Addr, err)
return err
@ -361,7 +361,7 @@ func (c *Client) SnapshotRPC(args *structs.SnapshotRequest, in io.Reader, out io
// Request the operation.
var reply structs.SnapshotResponse
snap, err := SnapshotRPC(c.connPool, c.config.Datacenter, server.Addr, args, in, &reply)
snap, err := SnapshotRPC(c.connPool, c.config.Datacenter, server.Addr, server.UseTLS, args, in, &reply)
if err != nil {
return err
}

View File

@ -129,12 +129,15 @@ type Config struct {
// must match a provided certificate authority. This can be used to force client auth.
VerifyIncoming bool
// VerifyOutgoing is used to verify the authenticity of outgoing connections.
// VerifyOutgoing is used to force verification of the authenticity of outgoing connections.
// This means that TLS requests are used, and TCP requests are not made. TLS connections
// must match a provided certificate authority. This is used to verify authenticity of
// server nodes.
// must match a provided certificate authority.
VerifyOutgoing bool
// UseTLS is used to enable TLS for outgoing connections to other TLS-capable Consul
// servers. This doesn't imply any verification, it only enables TLS if possible.
UseTLS bool
// VerifyServerHostname is used to enable hostname verification of servers. This
// ensures that the certificate presented is valid for server.<datacenter>.<domain>.
// This prevents a compromised client from being restarted as a server, and then
@ -439,6 +442,7 @@ func (c *Config) tlsConfig() *tlsutil.Config {
VerifyIncoming: c.VerifyIncoming,
VerifyOutgoing: c.VerifyOutgoing,
VerifyServerHostname: c.VerifyServerHostname,
UseTLS: c.UseTLS,
CAFile: c.CAFile,
CAPath: c.CAPath,
CertFile: c.CertFile,

View File

@ -144,6 +144,9 @@ type ConnPool struct {
// TLS wrapper
tlsWrap tlsutil.DCWrapper
// forceTLS is used to enforce outgoing TLS verification
forceTLS bool
// Used to indicate the pool is shutdown
shutdown bool
shutdownCh chan struct{}
@ -154,7 +157,7 @@ type ConnPool struct {
// Set maxTime to 0 to disable reaping. maxStreams is used to control
// the number of idle streams allowed.
// If TLS settings are provided outgoing connections use TLS.
func NewPool(src *net.TCPAddr, logOutput io.Writer, maxTime time.Duration, maxStreams int, tlsWrap tlsutil.DCWrapper) *ConnPool {
func NewPool(src *net.TCPAddr, logOutput io.Writer, maxTime time.Duration, maxStreams int, tlsWrap tlsutil.DCWrapper, forceTLS bool) *ConnPool {
pool := &ConnPool{
src: src,
logOutput: logOutput,
@ -163,6 +166,7 @@ func NewPool(src *net.TCPAddr, logOutput io.Writer, maxTime time.Duration, maxSt
pool: make(map[string]*Conn),
limiter: make(map[string]chan struct{}),
tlsWrap: tlsWrap,
forceTLS: forceTLS,
shutdownCh: make(chan struct{}),
}
if maxTime > 0 {
@ -193,7 +197,7 @@ func (p *ConnPool) Shutdown() error {
// wait for an existing connection attempt to finish, if one if in progress,
// and will return that one if it succeeds. If all else fails, it will return a
// newly-created connection and add it to the pool.
func (p *ConnPool) acquire(dc string, addr net.Addr, version int) (*Conn, error) {
func (p *ConnPool) acquire(dc string, addr net.Addr, version int, useTLS bool) (*Conn, error) {
addrStr := addr.String()
// Check to see if there's a pooled connection available. This is up
@ -222,7 +226,7 @@ func (p *ConnPool) acquire(dc string, addr net.Addr, version int) (*Conn, error)
// If we are the lead thread, make the new connection and then wake
// everybody else up to see if we got it.
if isLeadThread {
c, err := p.getNewConn(dc, addr, version)
c, err := p.getNewConn(dc, addr, version, useTLS)
p.Lock()
delete(p.limiter, addrStr)
close(wait)
@ -267,7 +271,7 @@ type HalfCloser interface {
// DialTimeout is used to establish a raw connection to the given server, with a
// given connection timeout.
func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration) (net.Conn, HalfCloser, error) {
func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration, useTLS bool) (net.Conn, HalfCloser, error) {
// Try to dial the conn
d := &net.Dialer{LocalAddr: p.src, Timeout: timeout}
conn, err := d.Dial("tcp", addr.String())
@ -284,7 +288,7 @@ func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration)
}
// Check if TLS is enabled
if p.tlsWrap != nil {
if (useTLS || p.forceTLS) && p.tlsWrap != nil {
// Switch the connection into TLS mode
if _, err := conn.Write([]byte{byte(rpcTLS)}); err != nil {
conn.Close()
@ -304,9 +308,9 @@ func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration)
}
// getNewConn is used to return a new connection
func (p *ConnPool) getNewConn(dc string, addr net.Addr, version int) (*Conn, error) {
func (p *ConnPool) getNewConn(dc string, addr net.Addr, version int, useTLS bool) (*Conn, error) {
// Get a new, raw connection.
conn, _, err := p.DialTimeout(dc, addr, defaultDialTimeout)
conn, _, err := p.DialTimeout(dc, addr, defaultDialTimeout, useTLS)
if err != nil {
return nil, err
}
@ -372,11 +376,11 @@ func (p *ConnPool) releaseConn(conn *Conn) {
}
// getClient is used to get a usable client for an address and protocol version
func (p *ConnPool) getClient(dc string, addr net.Addr, version int) (*Conn, *StreamClient, error) {
func (p *ConnPool) getClient(dc string, addr net.Addr, version int, useTLS bool) (*Conn, *StreamClient, error) {
retries := 0
START:
// Try to get a conn first
conn, err := p.acquire(dc, addr, version)
conn, err := p.acquire(dc, addr, version, useTLS)
if err != nil {
return nil, nil, fmt.Errorf("failed to get conn: %v", err)
}
@ -398,9 +402,9 @@ START:
}
// RPC is used to make an RPC call to a remote host
func (p *ConnPool) RPC(dc string, addr net.Addr, version int, method string, args interface{}, reply interface{}) error {
func (p *ConnPool) RPC(dc string, addr net.Addr, version int, method string, useTLS bool, args interface{}, reply interface{}) error {
// Get a usable client
conn, sc, err := p.getClient(dc, addr, version)
conn, sc, err := p.getClient(dc, addr, version, useTLS)
if err != nil {
return fmt.Errorf("rpc error: %v", err)
}
@ -423,7 +427,7 @@ func (p *ConnPool) RPC(dc string, addr net.Addr, version int, method string, arg
// returns true if healthy, false if an error occurred
func (p *ConnPool) PingConsulServer(s *agent.Server) (bool, error) {
// Get a usable client
conn, sc, err := p.getClient(s.Datacenter, s.Addr, s.Version)
conn, sc, err := p.getClient(s.Datacenter, s.Addr, s.Version, s.UseTLS)
if err != nil {
return false, err
}

View File

@ -29,18 +29,23 @@ type RaftLayer struct {
closed bool
closeCh chan struct{}
closeLock sync.Mutex
// tlsFunc is a callback to determine whether to use TLS for connecting to
// a given Raft server
tlsFunc func(raft.ServerAddress) bool
}
// NewRaftLayer is used to initialize a new RaftLayer which can
// be used as a StreamLayer for Raft. If a tlsConfig is provided,
// then the connection will use TLS.
func NewRaftLayer(src, addr net.Addr, tlsWrap tlsutil.Wrapper) *RaftLayer {
func NewRaftLayer(src, addr net.Addr, tlsWrap tlsutil.Wrapper, tlsFunc func(raft.ServerAddress) bool) *RaftLayer {
layer := &RaftLayer{
src: src,
addr: addr,
connCh: make(chan net.Conn),
tlsWrap: tlsWrap,
closeCh: make(chan struct{}),
tlsFunc: tlsFunc,
}
return layer
}
@ -93,7 +98,7 @@ func (l *RaftLayer) Dial(address raft.ServerAddress, timeout time.Duration) (net
}
// Check for tls mode
if l.tlsWrap != nil {
if l.tlsFunc(address) && l.tlsWrap != nil {
// Switch the connection into TLS mode
if _, err := conn.Write([]byte{byte(rpcTLS)}); err != nil {
conn.Close()

View File

@ -262,7 +262,7 @@ func (s *Server) forwardLeader(server *agent.Server, method string, args interfa
if server == nil {
return structs.ErrNoLeader
}
return s.connPool.RPC(s.config.Datacenter, server.Addr, server.Version, method, args, reply)
return s.connPool.RPC(s.config.Datacenter, server.Addr, server.Version, method, server.UseTLS, args, reply)
}
// forwardDC is used to forward an RPC call to a remote DC, or fail if no servers
@ -274,7 +274,7 @@ func (s *Server) forwardDC(method, dc string, args interface{}, reply interface{
}
metrics.IncrCounter([]string{"consul", "rpc", "cross-dc", dc}, 1)
if err := s.connPool.RPC(dc, server.Addr, server.Version, method, args, reply); err != nil {
if err := s.connPool.RPC(dc, server.Addr, server.Version, method, server.UseTLS, args, reply); err != nil {
manager.NotifyFailedServer(server)
s.logger.Printf("[ERR] consul: RPC failed to server %s in DC %q: %v", server.Addr, dc, err)
return err

View File

@ -199,7 +199,7 @@ func (s *Server) maybeBootstrap() {
// Retry with exponential backoff to get peer status from this server
for attempt := uint(0); attempt < maxPeerRetries; attempt++ {
if err := s.connPool.RPC(s.config.Datacenter, server.Addr, server.Version,
"Status.Peers", &struct{}{}, &peers); err != nil {
"Status.Peers", server.UseTLS, &struct{}{}, &peers); err != nil {
nextRetry := time.Duration((1 << attempt) * peerRetryBase)
s.logger.Printf("[ERR] consul: Failed to confirm peer status for %s: %v. Retrying in "+
"%v...", server.Name, err, nextRetry.String())

View File

@ -234,6 +234,11 @@ func NewServer(config *Config) (*Server, error) {
}
logger := log.New(config.LogOutput, "", log.LstdFlags)
// Check if TLS is enabled
if config.CAFile != "" || config.CAPath != "" {
config.UseTLS = true
}
// Create the TLS wrapper for outgoing connections.
tlsConf := config.tlsConfig()
tlsWrap, err := tlsConf.OutgoingTLSWrapper()
@ -261,7 +266,7 @@ func NewServer(config *Config) (*Server, error) {
autopilotRemoveDeadCh: make(chan struct{}),
autopilotShutdownCh: make(chan struct{}),
config: config,
connPool: NewPool(config.RPCSrcAddr, config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap),
connPool: NewPool(config.RPCSrcAddr, config.LogOutput, serverRPCCache, serverMaxStreams, tlsWrap, config.VerifyOutgoing),
eventChLAN: make(chan serf.Event, 256),
eventChWAN: make(chan serf.Event, 256),
localConsuls: make(map[raft.ServerAddress]*agent.Server),
@ -393,6 +398,9 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string, w
if s.config.NonVoter {
conf.Tags["nonvoter"] = "1"
}
if s.config.UseTLS {
conf.Tags["use_tls"] = "1"
}
conf.MemberlistConfig.LogOutput = s.config.LogOutput
conf.LogOutput = s.config.LogOutput
conf.EventCh = ch
@ -626,7 +634,24 @@ func (s *Server) setupRPC(tlsWrap tlsutil.DCWrapper) error {
// Provide a DC specific wrapper. Raft replication is only
// ever done in the same datacenter, so we can provide it as a constant.
wrapper := tlsutil.SpecificDC(s.config.Datacenter, tlsWrap)
s.raftLayer = NewRaftLayer(s.config.RPCSrcAddr, s.config.RPCAdvertise, wrapper)
// Define a callback for determining whether to wrap a connection with TLS
tlsFunc := func(address raft.ServerAddress) bool {
if s.config.VerifyOutgoing {
return true
}
s.localLock.RLock()
server, ok := s.localConsuls[address]
s.localLock.RUnlock()
if !ok {
return false
}
return server.UseTLS
}
s.raftLayer = NewRaftLayer(s.config.RPCSrcAddr, s.config.RPCAdvertise, wrapper, tlsFunc)
return nil
}

View File

@ -10,6 +10,7 @@ import (
"testing"
"time"
"github.com/hashicorp/consul/consul/agent"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/testutil/retry"
"github.com/hashicorp/consul/types"
@ -578,3 +579,111 @@ func TestServer_Encrypted(t *testing.T) {
t.Fatalf("should be encrypted")
}
}
func testVerifyRPC(s1, s2 *Server, t *testing.T) (bool, error) {
// Try to join
addr := fmt.Sprintf("127.0.0.1:%d",
s1.config.SerfLANConfig.MemberlistConfig.BindPort)
if _, err := s2.JoinLAN([]string{addr}); err != nil {
t.Fatalf("err: %v", err)
}
// Check the members
retry.Run(t, func(r *retry.R) { r.Check(wantPeers(s1, 2)) })
// Have s2 make an RPC call to s1
s2.localLock.RLock()
var leader *agent.Server
for _, server := range s2.localConsuls {
if server.Name == s1.config.NodeName {
leader = server
}
}
s2.localLock.RUnlock()
return s2.connPool.PingConsulServer(leader)
}
func TestServer_TLSToNoTLS(t *testing.T) {
// Set up a server with no TLS configured
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Add a second server with TLS configured
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Bootstrap = false
c.CAFile = "../test/client_certs/rootca.crt"
c.CertFile = "../test/client_certs/server.crt"
c.KeyFile = "../test/client_certs/server.key"
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
success, err := testVerifyRPC(s1, s2, t)
if err != nil {
t.Fatal(err)
}
if !success {
t.Fatalf("bad: %v", success)
}
}
func TestServer_TLSForceOutgoingToNoTLS(t *testing.T) {
// Set up a server with no TLS configured
dir1, s1 := testServer(t)
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Add a second server with TLS and VerifyOutgoing set
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Bootstrap = false
c.CAFile = "../test/client_certs/rootca.crt"
c.CertFile = "../test/client_certs/server.crt"
c.KeyFile = "../test/client_certs/server.key"
c.VerifyOutgoing = true
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
_, err := testVerifyRPC(s1, s2, t)
if err == nil || !strings.Contains(err.Error(), "remote error: tls") {
t.Fatalf("should fail")
}
}
func TestServer_TLSToFullVerify(t *testing.T) {
// Set up a server with TLS and VerifyIncoming set
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.CAFile = "../test/client_certs/rootca.crt"
c.CertFile = "../test/client_certs/server.crt"
c.KeyFile = "../test/client_certs/server.key"
c.VerifyIncoming = true
c.VerifyOutgoing = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Add a second server with TLS configured
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Bootstrap = false
c.CAFile = "../test/client_certs/rootca.crt"
c.CertFile = "../test/client_certs/server.crt"
c.KeyFile = "../test/client_certs/server.key"
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
success, err := testVerifyRPC(s1, s2, t)
if err != nil {
t.Fatal(err)
}
if !success {
t.Fatalf("bad: %v", success)
}
}

View File

@ -35,7 +35,7 @@ func (s *Server) dispatchSnapshotRequest(args *structs.SnapshotRequest, in io.Re
return nil, structs.ErrNoDCPath
}
snap, err := SnapshotRPC(s.connPool, dc, server.Addr, args, in, reply)
snap, err := SnapshotRPC(s.connPool, dc, server.Addr, server.UseTLS, args, in, reply)
if err != nil {
manager.NotifyFailedServer(server)
return nil, err
@ -50,7 +50,7 @@ func (s *Server) dispatchSnapshotRequest(args *structs.SnapshotRequest, in io.Re
if server == nil {
return nil, structs.ErrNoLeader
}
return SnapshotRPC(s.connPool, args.Datacenter, server.Addr, args, in, reply)
return SnapshotRPC(s.connPool, args.Datacenter, server.Addr, server.UseTLS, args, in, reply)
}
}
@ -187,10 +187,10 @@ RESPOND:
// the streaming output (for a snapshot). If the reply contains an error, this
// will always return an error as well, so you don't need to check the error
// inside the filled-in reply.
func SnapshotRPC(pool *ConnPool, dc string, addr net.Addr,
func SnapshotRPC(pool *ConnPool, dc string, addr net.Addr, useTLS bool,
args *structs.SnapshotRequest, in io.Reader, reply *structs.SnapshotResponse) (io.ReadCloser, error) {
conn, hc, err := pool.DialTimeout(dc, addr, 10*time.Second)
conn, hc, err := pool.DialTimeout(dc, addr, 10*time.Second, useTLS)
if err != nil {
return nil, err
}

View File

@ -44,7 +44,7 @@ func verifySnapshot(t *testing.T, s *Server, dc, token string) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
snap, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr,
snap, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -116,7 +116,7 @@ func verifySnapshot(t *testing.T, s *Server, dc, token string) {
// Restore the snapshot.
args.Op = structs.SnapshotRestore
restore, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr,
restore, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
&args, snap, &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -187,7 +187,7 @@ func TestSnapshot_LeaderState(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
snap, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr,
snap, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -220,7 +220,7 @@ func TestSnapshot_LeaderState(t *testing.T) {
// Restore the snapshot.
args.Op = structs.SnapshotRestore
restore, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr,
restore, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
&args, snap, &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -257,7 +257,7 @@ func TestSnapshot_ACLDeny(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr,
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err == nil || !strings.Contains(err.Error(), permissionDenied) {
t.Fatalf("err: %v", err)
@ -271,7 +271,7 @@ func TestSnapshot_ACLDeny(t *testing.T) {
Op: structs.SnapshotRestore,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr,
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err == nil || !strings.Contains(err.Error(), permissionDenied) {
t.Fatalf("err: %v", err)
@ -358,7 +358,7 @@ func TestSnapshot_AllowStale(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr,
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err == nil || !strings.Contains(err.Error(), structs.ErrNoLeader.Error()) {
t.Fatalf("err: %v", err)
@ -375,7 +375,7 @@ func TestSnapshot_AllowStale(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr,
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err == nil || !strings.Contains(err.Error(), "Raft error when taking snapshot") {
t.Fatalf("err: %v", err)

View File

@ -41,7 +41,7 @@ func NewStatsFetcher(logger *log.Logger, pool *ConnPool, datacenter string) *Sta
func (f *StatsFetcher) fetch(server *agent.Server, replyCh chan *structs.ServerStats) {
var args struct{}
var reply structs.ServerStats
err := f.pool.RPC(f.datacenter, server.Addr, server.Version, "Status.RaftStats", &args, &reply)
err := f.pool.RPC(f.datacenter, server.Addr, server.Version, "Status.RaftStats", server.UseTLS, &args, &reply)
if err != nil {
f.logger.Printf("[WARN] consul: error getting server health from %q: %v",
server.Name, err)

View File

@ -49,6 +49,9 @@ type Config struct {
// existing clients.
VerifyServerHostname bool
// UseTLS is used to enable outgoing TLS connections to Consul servers.
UseTLS bool
// CAFile is a path to a certificate authority file. This is used with VerifyIncoming
// or VerifyOutgoing to verify the TLS connection.
CAFile string
@ -126,7 +129,7 @@ func (c *Config) OutgoingTLSConfig() (*tls.Config, error) {
if c.VerifyServerHostname {
c.VerifyOutgoing = true
}
if !c.VerifyOutgoing {
if !c.UseTLS && !c.VerifyOutgoing {
return nil, nil
}
// Create the tlsConfig

View File

@ -70,10 +70,10 @@ and [`verify_incoming`](/docs/agent/options.html#verify_incoming) options, respe
If [`verify_outgoing`](/docs/agent/options.html#verify_outgoing) is set, agents verify the
authenticity of Consul for outgoing connections. Server nodes must present a certificate signed
by the certificate authority present on all agents, set via the agent's
[`ca_file`](/docs/agent/options.html#ca_file) option. All server nodes must have an
appropriate key pair set using [`cert_file`](/docs/agent/options.html#cert_file) and
[`key_file`](/docs/agent/options.html#key_file).
by a common certificate authority present on all agents, set via the agent's
[`ca_file`](/docs/agent/options.html#ca_file) and [`ca_path`](/docs/agent/options.html#ca_path)
options. All server nodes must have an appropriate key pair set using [`cert_file`]
(/docs/agent/options.html#cert_file) and [`key_file`](/docs/agent/options.html#key_file).
If [`verify_server_hostname`](/docs/agent/options.html#verify_server_hostname) is set, then
outgoing connections perform hostname verification. All servers must have a certificate
@ -92,3 +92,21 @@ also disallow any non-TLS connections. To force clients to use TLS,
TLS is used to secure the RPC calls between agents, but gossip between nodes is done over UDP
and is secured using a symmetric key. See above for enabling gossip encryption.
## Configuring TLS on an existing cluster
As of version 0.8.3, Consul supports migrating to TLS-encrypted traffic on a running cluster
without downtime. This process assumes a starting point with no TLS settings configured, and involves
an intermediate step in order to get to full TLS encryption:
1. Generate the necessary keys/certs and set the `ca_file`/`ca_path`, `cert_file`, and `key_file`
settings in the configuration for each agent. Make sure the `verify_outgoing` and `verify_incoming`
options are set to `false`. HTTPS for the API can be enabled at this point by
setting the [`https`](/docs/agent/options.html#http_port) port.
2. Perform a rolling restart of each agent in the cluster. After this step, TLS should be enabled
everywhere but the agents will not yet be enforcing TLS.
3. Change the `verify_incoming` and `verify_outgoing` settings (as well as `verify_server_hostname`
if applicable) to `true`.
4. Perform another rolling restart of each agent in the cluster.
At this point, full TLS encryption for RPC communication should be enabled.