From 5bab68b9bb89405fa3a13279499c0e7326918686 Mon Sep 17 00:00:00 2001 From: Kyle Havlovitz Date: Wed, 10 May 2017 14:25:48 -0700 Subject: [PATCH] Add a path for transitioning to TLS on an existing cluster (#3001) Fixes #1705 --- command/agent/agent.go | 3 + consul/agent/server.go | 6 + consul/agent/server_test.go | 4 + consul/client.go | 6 +- consul/config.go | 10 +- consul/pool.go | 28 +++-- consul/raft_rpc.go | 9 +- consul/rpc.go | 4 +- consul/serf.go | 2 +- consul/server.go | 29 ++++- consul/server_test.go | 109 ++++++++++++++++++ consul/snapshot_endpoint.go | 8 +- consul/snapshot_endpoint_test.go | 16 +-- consul/stats_fetcher.go | 2 +- tlsutil/config.go | 5 +- .../docs/agent/encryption.html.markdown | 26 ++++- 16 files changed, 224 insertions(+), 43 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index ba15e3851a07..b9b4157541c4 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -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 diff --git a/consul/agent/server.go b/consul/agent/server.go index 42ea27210ce3..0c51424c63c7 100644 --- a/consul/agent/server.go +++ b/consul/agent/server.go @@ -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 } diff --git a/consul/agent/server_test.go b/consul/agent/server_test.go index 1ea6d674a461..30365599c01c 100644 --- a/consul/agent/server_test.go +++ b/consul/agent/server_test.go @@ -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) diff --git a/consul/client.go b/consul/client.go index ff5fd66510df..82723872f2ec 100644 --- a/consul/client.go +++ b/consul/client.go @@ -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 } diff --git a/consul/config.go b/consul/config.go index 7b677accece3..4b27a0b9837b 100644 --- a/consul/config.go +++ b/consul/config.go @@ -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... // 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, diff --git a/consul/pool.go b/consul/pool.go index 14903f416cd1..9dde11e6e997 100644 --- a/consul/pool.go +++ b/consul/pool.go @@ -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 } diff --git a/consul/raft_rpc.go b/consul/raft_rpc.go index 4cc4823c6c4c..461a21411aca 100644 --- a/consul/raft_rpc.go +++ b/consul/raft_rpc.go @@ -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() diff --git a/consul/rpc.go b/consul/rpc.go index 3b9849685580..eef78d9f4710 100644 --- a/consul/rpc.go +++ b/consul/rpc.go @@ -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 diff --git a/consul/serf.go b/consul/serf.go index 87b32a0a56ff..d65eabbffca4 100644 --- a/consul/serf.go +++ b/consul/serf.go @@ -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()) diff --git a/consul/server.go b/consul/server.go index a879d02afb26..6f61f5a6f3e4 100644 --- a/consul/server.go +++ b/consul/server.go @@ -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 } diff --git a/consul/server_test.go b/consul/server_test.go index 3377a063808e..3da390906b5f 100644 --- a/consul/server_test.go +++ b/consul/server_test.go @@ -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) + } +} diff --git a/consul/snapshot_endpoint.go b/consul/snapshot_endpoint.go index 3c6f13002177..b29e873f5051 100644 --- a/consul/snapshot_endpoint.go +++ b/consul/snapshot_endpoint.go @@ -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 } diff --git a/consul/snapshot_endpoint_test.go b/consul/snapshot_endpoint_test.go index ff8885a30ea0..cb3254f414b7 100644 --- a/consul/snapshot_endpoint_test.go +++ b/consul/snapshot_endpoint_test.go @@ -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) diff --git a/consul/stats_fetcher.go b/consul/stats_fetcher.go index 74611c419aee..209411a5d368 100644 --- a/consul/stats_fetcher.go +++ b/consul/stats_fetcher.go @@ -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) diff --git a/tlsutil/config.go b/tlsutil/config.go index b2224b843d70..415f66e25176 100644 --- a/tlsutil/config.go +++ b/tlsutil/config.go @@ -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 diff --git a/website/source/docs/agent/encryption.html.markdown b/website/source/docs/agent/encryption.html.markdown index d433709dd747..6f182ee32073 100644 --- a/website/source/docs/agent/encryption.html.markdown +++ b/website/source/docs/agent/encryption.html.markdown @@ -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. \ No newline at end of file