open-consul/agent/consul/rpc_test.go

735 lines
21 KiB
Go
Raw Normal View History

package consul
import (
"bytes"
wan federation via mesh gateways (#6884) This is like a Möbius strip of code due to the fact that low-level components (serf/memberlist) are connected to high-level components (the catalog and mesh-gateways) in a twisty maze of references which make it hard to dive into. With that in mind here's a high level summary of what you'll find in the patch: There are several distinct chunks of code that are affected: * new flags and config options for the server * retry join WAN is slightly different * retry join code is shared to discover primary mesh gateways from secondary datacenters * because retry join logic runs in the *agent* and the results of that operation for primary mesh gateways are needed in the *server* there are some methods like `RefreshPrimaryGatewayFallbackAddresses` that must occur at multiple layers of abstraction just to pass the data down to the right layer. * new cache type `FederationStateListMeshGatewaysName` for use in `proxycfg/xds` layers * the function signature for RPC dialing picked up a new required field (the node name of the destination) * several new RPCs for manipulating a FederationState object: `FederationState:{Apply,Get,List,ListMeshGateways}` * 3 read-only internal APIs for debugging use to invoke those RPCs from curl * raft and fsm changes to persist these FederationStates * replication for FederationStates as they are canonically stored in the Primary and replicated to the Secondaries. * a special derivative of anti-entropy that runs in secondaries to snapshot their local mesh gateway `CheckServiceNodes` and sync them into their upstream FederationState in the primary (this works in conjunction with the replication to distribute addresses for all mesh gateways in all DCs to all other DCs) * a "gateway locator" convenience object to make use of this data to choose the addresses of gateways to use for any given RPC or gossip operation to a remote DC. This gets data from the "retry join" logic in the agent and also directly calls into the FSM. * RPC (`:8300`) on the server sniffs the first byte of a new connection to determine if it's actually doing native TLS. If so it checks the ALPN header for protocol determination (just like how the existing system uses the type-byte marker). * 2 new kinds of protocols are exclusively decoded via this native TLS mechanism: one for ferrying "packet" operations (udp-like) from the gossip layer and one for "stream" operations (tcp-like). The packet operations re-use sockets (using length-prefixing) to cut down on TLS re-negotiation overhead. * the server instances specially wrap the `memberlist.NetTransport` when running with gateway federation enabled (in a `wanfed.Transport`). The general gist is that if it tries to dial a node in the SAME datacenter (deduced by looking at the suffix of the node name) there is no change. If dialing a DIFFERENT datacenter it is wrapped up in a TLS+ALPN blob and sent through some mesh gateways to eventually end up in a server's :8300 port. * a new flag when launching a mesh gateway via `consul connect envoy` to indicate that the servers are to be exposed. This sets a special service meta when registering the gateway into the catalog. * `proxycfg/xds` notice this metadata blob to activate additional watches for the FederationState objects as well as the location of all of the consul servers in that datacenter. * `xds:` if the extra metadata is in place additional clusters are defined in a DC to bulk sink all traffic to another DC's gateways. For the current datacenter we listen on a wildcard name (`server.<dc>.consul`) that load balances all servers as well as one mini-cluster per node (`<node>.server.<dc>.consul`) * the `consul tls cert create` command got a new flag (`-node`) to help create an additional SAN in certs that can be used with this flavor of federation.
2020-03-09 20:59:02 +00:00
"encoding/binary"
"math"
"net"
"os"
"strings"
wan federation via mesh gateways (#6884) This is like a Möbius strip of code due to the fact that low-level components (serf/memberlist) are connected to high-level components (the catalog and mesh-gateways) in a twisty maze of references which make it hard to dive into. With that in mind here's a high level summary of what you'll find in the patch: There are several distinct chunks of code that are affected: * new flags and config options for the server * retry join WAN is slightly different * retry join code is shared to discover primary mesh gateways from secondary datacenters * because retry join logic runs in the *agent* and the results of that operation for primary mesh gateways are needed in the *server* there are some methods like `RefreshPrimaryGatewayFallbackAddresses` that must occur at multiple layers of abstraction just to pass the data down to the right layer. * new cache type `FederationStateListMeshGatewaysName` for use in `proxycfg/xds` layers * the function signature for RPC dialing picked up a new required field (the node name of the destination) * several new RPCs for manipulating a FederationState object: `FederationState:{Apply,Get,List,ListMeshGateways}` * 3 read-only internal APIs for debugging use to invoke those RPCs from curl * raft and fsm changes to persist these FederationStates * replication for FederationStates as they are canonically stored in the Primary and replicated to the Secondaries. * a special derivative of anti-entropy that runs in secondaries to snapshot their local mesh gateway `CheckServiceNodes` and sync them into their upstream FederationState in the primary (this works in conjunction with the replication to distribute addresses for all mesh gateways in all DCs to all other DCs) * a "gateway locator" convenience object to make use of this data to choose the addresses of gateways to use for any given RPC or gossip operation to a remote DC. This gets data from the "retry join" logic in the agent and also directly calls into the FSM. * RPC (`:8300`) on the server sniffs the first byte of a new connection to determine if it's actually doing native TLS. If so it checks the ALPN header for protocol determination (just like how the existing system uses the type-byte marker). * 2 new kinds of protocols are exclusively decoded via this native TLS mechanism: one for ferrying "packet" operations (udp-like) from the gossip layer and one for "stream" operations (tcp-like). The packet operations re-use sockets (using length-prefixing) to cut down on TLS re-negotiation overhead. * the server instances specially wrap the `memberlist.NetTransport` when running with gateway federation enabled (in a `wanfed.Transport`). The general gist is that if it tries to dial a node in the SAME datacenter (deduced by looking at the suffix of the node name) there is no change. If dialing a DIFFERENT datacenter it is wrapped up in a TLS+ALPN blob and sent through some mesh gateways to eventually end up in a server's :8300 port. * a new flag when launching a mesh gateway via `consul connect envoy` to indicate that the servers are to be exposed. This sets a special service meta when registering the gateway into the catalog. * `proxycfg/xds` notice this metadata blob to activate additional watches for the FederationState objects as well as the location of all of the consul servers in that datacenter. * `xds:` if the extra metadata is in place additional clusters are defined in a DC to bulk sink all traffic to another DC's gateways. For the current datacenter we listen on a wildcard name (`server.<dc>.consul`) that load balances all servers as well as one mini-cluster per node (`<node>.server.<dc>.consul`) * the `consul tls cert create` command got a new flag (`-node`) to help create an additional SAN in certs that can be used with this flavor of federation.
2020-03-09 20:59:02 +00:00
"sync"
"testing"
"time"
pkg refactor command/agent/* -> agent/* command/consul/* -> agent/consul/* command/agent/command{,_test}.go -> command/agent{,_test}.go command/base/command.go -> command/base.go command/base/* -> command/* commands.go -> command/commands.go The script which did the refactor is: ( cd $GOPATH/src/github.com/hashicorp/consul git mv command/agent/command.go command/agent.go git mv command/agent/command_test.go command/agent_test.go git mv command/agent/flag_slice_value{,_test}.go command/ git mv command/agent . git mv command/base/command.go command/base.go git mv command/base/config_util{,_test}.go command/ git mv commands.go command/ git mv consul agent rmdir command/base/ gsed -i -e 's|package agent|package command|' command/agent{,_test}.go gsed -i -e 's|package agent|package command|' command/flag_slice_value{,_test}.go gsed -i -e 's|package base|package command|' command/base.go command/config_util{,_test}.go gsed -i -e 's|package main|package command|' command/commands.go gsed -i -e 's|base.Command|BaseCommand|' command/commands.go gsed -i -e 's|agent.Command|AgentCommand|' command/commands.go gsed -i -e 's|\tCommand:|\tBaseCommand:|' command/commands.go gsed -i -e 's|base\.||' command/commands.go gsed -i -e 's|command\.||' command/commands.go gsed -i -e 's|command|c|' main.go gsed -i -e 's|range Commands|range command.Commands|' main.go gsed -i -e 's|Commands: Commands|Commands: command.Commands|' main.go gsed -i -e 's|base\.BoolValue|BoolValue|' command/operator_autopilot_set.go gsed -i -e 's|base\.DurationValue|DurationValue|' command/operator_autopilot_set.go gsed -i -e 's|base\.StringValue|StringValue|' command/operator_autopilot_set.go gsed -i -e 's|base\.UintValue|UintValue|' command/operator_autopilot_set.go gsed -i -e 's|\bCommand\b|BaseCommand|' command/base.go gsed -i -e 's|BaseCommand Options|Command Options|' command/base.go gsed -i -e 's|base.Command|BaseCommand|' command/*.go gsed -i -e 's|c\.Command|c.BaseCommand|g' command/*.go gsed -i -e 's|\tCommand:|\tBaseCommand:|' command/*_test.go gsed -i -e 's|base\.||' command/*_test.go gsed -i -e 's|\bCommand\b|AgentCommand|' command/agent{,_test}.go gsed -i -e 's|cmd.AgentCommand|cmd.BaseCommand|' command/agent.go gsed -i -e 's|cli.AgentCommand = new(Command)|cli.Command = new(AgentCommand)|' command/agent_test.go gsed -i -e 's|exec.AgentCommand|exec.Command|' command/agent_test.go gsed -i -e 's|exec.BaseCommand|exec.Command|' command/agent_test.go gsed -i -e 's|NewTestAgent|agent.NewTestAgent|' command/agent_test.go gsed -i -e 's|= TestConfig|= agent.TestConfig|' command/agent_test.go gsed -i -e 's|: RetryJoin|: agent.RetryJoin|' command/agent_test.go gsed -i -e 's|\.\./\.\./|../|' command/config_util_test.go gsed -i -e 's|\bverifyUniqueListeners|VerifyUniqueListeners|' agent/config{,_test}.go command/agent.go gsed -i -e 's|\bserfLANKeyring\b|SerfLANKeyring|g' agent/{agent,keyring,testagent}.go command/agent.go gsed -i -e 's|\bserfWANKeyring\b|SerfWANKeyring|g' agent/{agent,keyring,testagent}.go command/agent.go gsed -i -e 's|\bNewAgent\b|agent.New|g' command/agent{,_test}.go gsed -i -e 's|\bNewAgent|New|' agent/{acl_test,agent,testagent}.go gsed -i -e 's|\bAgent\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bBool\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bConfig\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bDefaultConfig\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bDevConfig\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bMergeConfig\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bReadConfigPaths\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bParseMetaPair\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bSerfLANKeyring\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|\bSerfWANKeyring\b|agent.&|g' command/agent{,_test}.go gsed -i -e 's|circonus\.agent|circonus|g' command/agent{,_test}.go gsed -i -e 's|logger\.agent|logger|g' command/agent{,_test}.go gsed -i -e 's|metrics\.agent|metrics|g' command/agent{,_test}.go gsed -i -e 's|// agent.Agent|// agent|' command/agent{,_test}.go gsed -i -e 's|a\.agent\.Config|a.Config|' command/agent{,_test}.go gsed -i -e 's|agent\.AppendSliceValue|AppendSliceValue|' command/{configtest,validate}.go gsed -i -e 's|consul/consul|agent/consul|' GNUmakefile gsed -i -e 's|\.\./test|../../test|' agent/consul/server_test.go # fix imports f=$(grep -rl 'github.com/hashicorp/consul/command/agent' * | grep '\.go') gsed -i -e 's|github.com/hashicorp/consul/command/agent|github.com/hashicorp/consul/agent|' $f goimports -w $f f=$(grep -rl 'github.com/hashicorp/consul/consul' * | grep '\.go') gsed -i -e 's|github.com/hashicorp/consul/consul|github.com/hashicorp/consul/agent/consul|' $f goimports -w $f goimports -w command/*.go main.go )
2017-06-09 22:28:28 +00:00
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/pool"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/go-memdb"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestRPC_NoLeader_Fail(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCHoldTimeout = 1 * time.Millisecond
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
arg := structs.RegisterRequest{
Datacenter: "dc1",
Node: "foo",
Address: "127.0.0.1",
}
var out struct{}
// Make sure we eventually fail with a no leader error, which we should
// see given the short timeout.
err := msgpackrpc.CallWithCodec(codec, "Catalog.Register", &arg, &out)
2017-05-31 06:59:41 +00:00
if err == nil || err.Error() != structs.ErrNoLeader.Error() {
t.Fatalf("bad: %v", err)
}
// Now make sure it goes through.
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
err = msgpackrpc.CallWithCodec(codec, "Catalog.Register", &arg, &out)
if err != nil {
t.Fatalf("bad: %v", err)
}
}
func TestRPC_NoLeader_Fail_on_stale_read(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCHoldTimeout = 1 * time.Millisecond
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
arg := structs.RegisterRequest{
Datacenter: "dc1",
Node: "foo",
Address: "127.0.0.1",
}
var out struct{}
// Make sure we eventually fail with a no leader error, which we should
// see given the short timeout.
err := msgpackrpc.CallWithCodec(codec, "Catalog.Register", &arg, &out)
if err == nil || err.Error() != structs.ErrNoLeader.Error() {
t.Fatalf("bad: %v", err)
}
// Until leader has never been known, stale should fail
getKeysReq := structs.KeyListRequest{
Datacenter: "dc1",
Prefix: "",
Seperator: "/",
QueryOptions: structs.QueryOptions{AllowStale: true},
}
var keyList structs.IndexedKeyList
if err := msgpackrpc.CallWithCodec(codec, "KVS.ListKeys", &getKeysReq, &keyList); err.Error() != structs.ErrNoLeader.Error() {
t.Fatalf("expected %v but got err: %v", structs.ErrNoLeader, err)
}
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
if err := msgpackrpc.CallWithCodec(codec, "KVS.ListKeys", &getKeysReq, &keyList); err != nil {
t.Fatalf("Did not expect any error but got err: %v", err)
}
}
func TestRPC_NoLeader_Retry(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCHoldTimeout = 10 * time.Second
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
arg := structs.RegisterRequest{
Datacenter: "dc1",
Node: "foo",
Address: "127.0.0.1",
}
var out struct{}
// This isn't sure-fire but tries to check that we don't have a
// leader going into the RPC, so we exercise the retry logic.
if ok, _ := s1.getLeader(); ok {
t.Fatalf("should not have a leader yet")
}
// The timeout is long enough to ride out any reasonable leader
// election.
err := msgpackrpc.CallWithCodec(codec, "Catalog.Register", &arg, &out)
if err != nil {
t.Fatalf("bad: %v", err)
}
}
type MockSink struct {
*bytes.Buffer
cancel bool
}
func (m *MockSink) ID() string {
return "Mock"
}
func (m *MockSink) Cancel() error {
m.cancel = true
return nil
}
func (m *MockSink) Close() error {
return nil
}
func TestRPC_blockingQuery(t *testing.T) {
t.Parallel()
dir, s := testServer(t)
defer os.RemoveAll(dir)
defer s.Shutdown()
require := require.New(t)
assert := assert.New(t)
// Perform a non-blocking query. Note that it's significant that the meta has
// a zero index in response - the implied opts.MinQueryIndex is also zero but
// this should not block still.
{
var opts structs.QueryOptions
var meta structs.QueryMeta
var calls int
2017-04-21 00:46:29 +00:00
fn := func(ws memdb.WatchSet, state *state.Store) error {
calls++
return nil
}
if err := s.blockingQuery(&opts, &meta, fn); err != nil {
t.Fatalf("err: %v", err)
}
if calls != 1 {
t.Fatalf("bad: %d", calls)
}
}
// Perform a blocking query that gets woken up and loops around once.
{
opts := structs.QueryOptions{
MinQueryIndex: 3,
}
var meta structs.QueryMeta
var calls int
2017-04-21 00:46:29 +00:00
fn := func(ws memdb.WatchSet, state *state.Store) error {
if calls == 0 {
meta.Index = 3
fakeCh := make(chan struct{})
close(fakeCh)
ws.Add(fakeCh)
} else {
meta.Index = 4
}
calls++
return nil
}
if err := s.blockingQuery(&opts, &meta, fn); err != nil {
t.Fatalf("err: %v", err)
}
if calls != 2 {
t.Fatalf("bad: %d", calls)
}
}
// Perform a blocking query that returns a zero index from blocking func (e.g.
// no state yet). This should still return an empty response immediately, but
// with index of 1 and then block on the next attempt. In one sense zero index
// is not really a valid response from a state method that is not an error but
// in practice a lot of state store operations do return it unless they
// explicitly special checks to turn 0 into 1. Often this is not caught or
// covered by tests but eventually when hit in the wild causes blocking
// clients to busy loop and burn CPU. This test ensure that blockingQuery
// systematically does the right thing to prevent future bugs like that.
{
opts := structs.QueryOptions{
MinQueryIndex: 0,
}
var meta structs.QueryMeta
var calls int
fn := func(ws memdb.WatchSet, state *state.Store) error {
if opts.MinQueryIndex > 0 {
// If client requested blocking, block forever. This is simulating
// waiting for the watched resource to be initialized/written to giving
// it a non-zero index. Note the timeout on the query options is relied
// on to stop the test taking forever.
fakeCh := make(chan struct{})
ws.Add(fakeCh)
}
meta.Index = 0
calls++
return nil
}
require.NoError(s.blockingQuery(&opts, &meta, fn))
assert.Equal(1, calls)
assert.Equal(uint64(1), meta.Index,
"expect fake index of 1 to force client to block on next update")
// Simulate client making next request
opts.MinQueryIndex = 1
opts.MaxQueryTime = 20 * time.Millisecond // Don't wait too long
// This time we should block even though the func returns index 0 still
t0 := time.Now()
require.NoError(s.blockingQuery(&opts, &meta, fn))
t1 := time.Now()
assert.Equal(2, calls)
assert.Equal(uint64(1), meta.Index,
"expect fake index of 1 to force client to block on next update")
assert.True(t1.Sub(t0) > 20*time.Millisecond,
"should have actually blocked waiting for timeout")
}
// Perform a query that blocks and gets interrupted when the state store
// is abandoned.
{
opts := structs.QueryOptions{
MinQueryIndex: 3,
}
var meta structs.QueryMeta
var calls int
2017-04-21 00:46:29 +00:00
fn := func(ws memdb.WatchSet, state *state.Store) error {
if calls == 0 {
meta.Index = 3
snap, err := s.fsm.Snapshot()
if err != nil {
t.Fatalf("err: %v", err)
}
defer snap.Release()
buf := bytes.NewBuffer(nil)
sink := &MockSink{buf, false}
if err := snap.Persist(sink); err != nil {
t.Fatalf("err: %v", err)
}
if err := s.fsm.Restore(sink); err != nil {
t.Fatalf("err: %v", err)
}
}
calls++
return nil
}
if err := s.blockingQuery(&opts, &meta, fn); err != nil {
t.Fatalf("err: %v", err)
}
if calls != 1 {
t.Fatalf("bad: %d", calls)
}
}
}
func TestRPC_ReadyForConsistentReads(t *testing.T) {
t.Parallel()
dir, s := testServerWithConfig(t, func(c *Config) {
c.RPCHoldTimeout = 2 * time.Millisecond
})
defer os.RemoveAll(dir)
defer s.Shutdown()
testrpc.WaitForLeader(t, s.RPC, "dc1")
if !s.isReadyForConsistentReads() {
t.Fatal("Server should be ready for consistent reads")
}
s.resetConsistentReadReady()
err := s.consistentRead()
2017-06-21 00:43:07 +00:00
if err.Error() != "Not ready to serve consistent reads" {
t.Fatal("Server should NOT be ready for consistent reads")
}
go func() {
time.Sleep(100 * time.Millisecond)
s.setConsistentReadReady()
}()
retry.Run(t, func(r *retry.R) {
if err := s.consistentRead(); err != nil {
r.Fatalf("Expected server to be ready for consistent reads, got error %v", err)
}
})
}
func TestRPC_MagicByteTimeout(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCHandshakeTimeout = 10 * time.Millisecond
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
// Connect to the server with bare TCP to simulate a malicious client trying
// to hold open resources.
addr := s1.config.RPCAdvertise
conn, err := net.DialTimeout("tcp", addr.String(), time.Second)
require.NoError(t, err)
defer conn.Close()
// Wait for more than the timeout. This is timing dependent so could fail if
// the CPU is super overloaded so the handler goroutine so I'm using a retry
// loop below to be sure but this feels like a pretty generous margin for
// error (10x the timeout and 100ms of scheduling time).
time.Sleep(100 * time.Millisecond)
// Set a read deadline on the Conn in case the timeout is not working we don't
// want the read below to block forever. Needs to be much longer than what we
// expect and the error should be different too.
conn.SetReadDeadline(time.Now().Add(3 * time.Second))
retry.Run(t, func(r *retry.R) {
// Sanity check the conn was closed by attempting to read from it (a write
// might not detect the close).
buf := make([]byte, 10)
_, err = conn.Read(buf)
require.Error(r, err)
require.Contains(r, err.Error(), "EOF")
})
}
func TestRPC_TLSHandshakeTimeout(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCHandshakeTimeout = 10 * time.Millisecond
c.UseTLS = true
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Alice.crt"
c.KeyFile = "../../test/hostname/Alice.key"
c.VerifyServerHostname = true
c.VerifyOutgoing = true
c.VerifyIncoming = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
// Connect to the server with TLS magic byte delivered on time
addr := s1.config.RPCAdvertise
conn, err := net.DialTimeout("tcp", addr.String(), time.Second)
require.NoError(t, err)
defer conn.Close()
// Write TLS byte to avoid being closed by either the (outer) first byte
// timeout or the fact that server requires TLS
_, err = conn.Write([]byte{pool.RPCTLS})
require.NoError(t, err)
// Wait for more than the timeout before we start a TLS handshake. This is
// timing dependent so could fail if the CPU is super overloaded so the
// handler goroutine so I'm using a retry loop below to be sure but this feels
// like a pretty generous margin for error (10x the timeout and 100ms of
// scheduling time).
time.Sleep(100 * time.Millisecond)
// Set a read deadline on the Conn in case the timeout is not working we don't
// want the read below to block forever. Needs to be much longer than what we
// expect and the error should be different too.
conn.SetReadDeadline(time.Now().Add(3 * time.Second))
retry.Run(t, func(r *retry.R) {
// Sanity check the conn was closed by attempting to read from it (a write
// might not detect the close).
buf := make([]byte, 10)
_, err = conn.Read(buf)
require.Error(r, err)
require.Contains(r, err.Error(), "EOF")
})
}
func TestRPC_PreventsTLSNesting(t *testing.T) {
t.Parallel()
cases := []struct {
name string
outerByte pool.RPCType
innerByte pool.RPCType
wantClose bool
}{
{
// Base case, sanity check normal RPC in TLS works
name: "RPC in TLS",
outerByte: pool.RPCTLS,
innerByte: pool.RPCConsul,
wantClose: false,
},
{
// Nested TLS-in-TLS
name: "TLS in TLS",
outerByte: pool.RPCTLS,
innerByte: pool.RPCTLS,
wantClose: true,
},
{
// Nested TLS-in-TLS
name: "TLS in Insecure TLS",
outerByte: pool.RPCTLSInsecure,
innerByte: pool.RPCTLS,
wantClose: true,
},
{
// Nested TLS-in-TLS
name: "Insecure TLS in TLS",
outerByte: pool.RPCTLS,
innerByte: pool.RPCTLSInsecure,
wantClose: true,
},
{
// Nested TLS-in-TLS
name: "Insecure TLS in Insecure TLS",
outerByte: pool.RPCTLSInsecure,
innerByte: pool.RPCTLSInsecure,
wantClose: true,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.UseTLS = true
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Alice.crt"
c.KeyFile = "../../test/hostname/Alice.key"
c.VerifyServerHostname = true
c.VerifyOutgoing = true
c.VerifyIncoming = false // saves us getting client cert setup
c.Domain = "consul"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
// Connect to the server with TLS magic byte delivered on time
addr := s1.config.RPCAdvertise
conn, err := net.DialTimeout("tcp", addr.String(), time.Second)
require.NoError(t, err)
defer conn.Close()
// Write Outer magic byte
_, err = conn.Write([]byte{byte(tc.outerByte)})
require.NoError(t, err)
// Start tls client
tlsWrap := s1.tlsConfigurator.OutgoingRPCWrapper()
tlsConn, err := tlsWrap("dc1", conn)
require.NoError(t, err)
// Write Inner magic byte
_, err = tlsConn.Write([]byte{byte(tc.innerByte)})
require.NoError(t, err)
if tc.wantClose {
// Allow up to a second for a read failure to indicate conn was closed by
// server.
conn.SetReadDeadline(time.Now().Add(1 * time.Second))
retry.Run(t, func(r *retry.R) {
// Sanity check the conn was closed by attempting to read from it (a
// write might not detect the close).
buf := make([]byte, 10)
_, err = tlsConn.Read(buf)
require.Error(r, err)
require.Contains(r, err.Error(), "EOF")
})
} else {
// Set a shorter read deadline that should typically be enough to detect
// immediate close but will also not make test hang forever. This
// positive case is mostly just a sanity check that the test code here
// is actually not failing just due to some other error in the way we
// setup TLS. It also sanity checks that we still allow valid TLS conns
// but if it produces possible false-positives in CI sometimes that's
// not such a huge deal - CI won't be brittle and it will have done it's
// job as a sanity check most of the time.
conn.SetReadDeadline(time.Now().Add(50 * time.Millisecond))
buf := make([]byte, 10)
_, err = tlsConn.Read(buf)
require.Error(t, err)
require.Contains(t, err.Error(), "i/o timeout")
}
})
}
}
func connectClient(t *testing.T, s1 *Server, mb pool.RPCType, useTLS, wantOpen bool, message string) net.Conn {
t.Helper()
addr := s1.config.RPCAdvertise
tlsWrap := s1.tlsConfigurator.OutgoingRPCWrapper()
conn, err := net.DialTimeout("tcp", addr.String(), time.Second)
require.NoError(t, err)
// Write magic byte so we aren't timed out
outerByte := mb
if useTLS {
outerByte = pool.RPCTLS
}
_, err = conn.Write([]byte{byte(outerByte)})
require.NoError(t, err)
if useTLS {
tlsConn, err := tlsWrap(s1.config.Datacenter, conn)
// Subtly, tlsWrap will NOT actually do a handshake in this case - it only
// does so for some configs, so even if the server closed the conn before
// handshake this won't fail and it's only when we attempt to read or write
// that we'll see the broken pipe.
require.NoError(t, err, "%s: wanted open conn, failed TLS handshake: %s",
message, err)
conn = tlsConn
// Write Inner magic byte
_, err = conn.Write([]byte{byte(mb)})
if !wantOpen {
// TLS Handshake will be done on this attempt to write and should fail
require.Error(t, err, "%s: wanted closed conn, TLS Handshake succeeded", message)
} else {
require.NoError(t, err, "%s: wanted open conn, failed writing inner magic byte: %s",
message, err)
}
}
// Check if the conn is in the state we want.
retry.Run(t, func(r *retry.R) {
// Don't wait around as server won't be sending data but the read will fail
// immediately if the conn is closed.
conn.SetReadDeadline(time.Now().Add(1 * time.Millisecond))
buf := make([]byte, 10)
_, err := conn.Read(buf)
require.Error(r, err)
if wantOpen {
require.Contains(r, err.Error(), "i/o timeout",
"%s: wanted an open conn (read timeout)", message)
} else {
if useTLS {
require.Error(r, err)
// TLS may fail during either read or write of the handshake so there
// are a few different errors that come up.
if !strings.Contains(err.Error(), "read: connection reset by peer") &&
!strings.Contains(err.Error(), "write: connection reset by peer") &&
!strings.Contains(err.Error(), "write: broken pipe") {
r.Fatalf("%s: wanted closed conn got err: %s", message, err)
}
} else {
require.Contains(r, err.Error(), "EOF", "%s: wanted a closed conn",
message)
}
}
})
return conn
}
func TestRPC_RPCMaxConnsPerClient(t *testing.T) {
t.Parallel()
cases := []struct {
name string
magicByte pool.RPCType
tlsEnabled bool
}{
{"RPC", pool.RPCMultiplexV2, false},
{"RPC TLS", pool.RPCMultiplexV2, true},
{"Raft", pool.RPCRaft, false},
{"Raft TLS", pool.RPCRaft, true},
}
for _, tc := range cases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.RPCMaxConnsPerClient = 2
if tc.tlsEnabled {
c.UseTLS = true
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Alice.crt"
c.KeyFile = "../../test/hostname/Alice.key"
c.VerifyServerHostname = true
c.VerifyOutgoing = true
c.VerifyIncoming = false // saves us getting client cert setup
c.Domain = "consul"
}
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
// Connect to the server with bare TCP
conn1 := connectClient(t, s1, tc.magicByte, tc.tlsEnabled, true, "conn1")
defer conn1.Close()
// Two conns should succeed
conn2 := connectClient(t, s1, tc.magicByte, tc.tlsEnabled, true, "conn2")
defer conn2.Close()
// Third should be closed byt the limiter
conn3 := connectClient(t, s1, tc.magicByte, tc.tlsEnabled, false, "conn3")
defer conn3.Close()
// If we close one of the earlier ones, we should be able to open another
addr := conn1.RemoteAddr()
conn1.Close()
retry.Run(t, func(r *retry.R) {
if n := s1.rpcConnLimiter.NumOpen(addr); n >= 2 {
r.Fatal("waiting for open conns to drop")
}
})
conn4 := connectClient(t, s1, tc.magicByte, tc.tlsEnabled, true, "conn4")
defer conn4.Close()
// Reload config with higher limit
newCfg := *s1.config
newCfg.RPCMaxConnsPerClient = 10
require.NoError(t, s1.ReloadConfig(&newCfg))
// Now another conn should be allowed
conn5 := connectClient(t, s1, tc.magicByte, tc.tlsEnabled, true, "conn5")
defer conn5.Close()
})
}
}
wan federation via mesh gateways (#6884) This is like a Möbius strip of code due to the fact that low-level components (serf/memberlist) are connected to high-level components (the catalog and mesh-gateways) in a twisty maze of references which make it hard to dive into. With that in mind here's a high level summary of what you'll find in the patch: There are several distinct chunks of code that are affected: * new flags and config options for the server * retry join WAN is slightly different * retry join code is shared to discover primary mesh gateways from secondary datacenters * because retry join logic runs in the *agent* and the results of that operation for primary mesh gateways are needed in the *server* there are some methods like `RefreshPrimaryGatewayFallbackAddresses` that must occur at multiple layers of abstraction just to pass the data down to the right layer. * new cache type `FederationStateListMeshGatewaysName` for use in `proxycfg/xds` layers * the function signature for RPC dialing picked up a new required field (the node name of the destination) * several new RPCs for manipulating a FederationState object: `FederationState:{Apply,Get,List,ListMeshGateways}` * 3 read-only internal APIs for debugging use to invoke those RPCs from curl * raft and fsm changes to persist these FederationStates * replication for FederationStates as they are canonically stored in the Primary and replicated to the Secondaries. * a special derivative of anti-entropy that runs in secondaries to snapshot their local mesh gateway `CheckServiceNodes` and sync them into their upstream FederationState in the primary (this works in conjunction with the replication to distribute addresses for all mesh gateways in all DCs to all other DCs) * a "gateway locator" convenience object to make use of this data to choose the addresses of gateways to use for any given RPC or gossip operation to a remote DC. This gets data from the "retry join" logic in the agent and also directly calls into the FSM. * RPC (`:8300`) on the server sniffs the first byte of a new connection to determine if it's actually doing native TLS. If so it checks the ALPN header for protocol determination (just like how the existing system uses the type-byte marker). * 2 new kinds of protocols are exclusively decoded via this native TLS mechanism: one for ferrying "packet" operations (udp-like) from the gossip layer and one for "stream" operations (tcp-like). The packet operations re-use sockets (using length-prefixing) to cut down on TLS re-negotiation overhead. * the server instances specially wrap the `memberlist.NetTransport` when running with gateway federation enabled (in a `wanfed.Transport`). The general gist is that if it tries to dial a node in the SAME datacenter (deduced by looking at the suffix of the node name) there is no change. If dialing a DIFFERENT datacenter it is wrapped up in a TLS+ALPN blob and sent through some mesh gateways to eventually end up in a server's :8300 port. * a new flag when launching a mesh gateway via `consul connect envoy` to indicate that the servers are to be exposed. This sets a special service meta when registering the gateway into the catalog. * `proxycfg/xds` notice this metadata blob to activate additional watches for the FederationState objects as well as the location of all of the consul servers in that datacenter. * `xds:` if the extra metadata is in place additional clusters are defined in a DC to bulk sink all traffic to another DC's gateways. For the current datacenter we listen on a wildcard name (`server.<dc>.consul`) that load balances all servers as well as one mini-cluster per node (`<node>.server.<dc>.consul`) * the `consul tls cert create` command got a new flag (`-node`) to help create an additional SAN in certs that can be used with this flavor of federation.
2020-03-09 20:59:02 +00:00
func TestRPC_readUint32(t *testing.T) {
cases := []struct {
name string
writeFn func(net.Conn)
readFn func(*testing.T, net.Conn)
}{
{
name: "timeouts irrelevant",
writeFn: func(conn net.Conn) {
_ = binary.Write(conn, binary.BigEndian, uint32(42))
_ = binary.Write(conn, binary.BigEndian, uint32(math.MaxUint32))
_ = binary.Write(conn, binary.BigEndian, uint32(1))
},
readFn: func(t *testing.T, conn net.Conn) {
t.Helper()
v, err := readUint32(conn, 5*time.Second)
require.NoError(t, err)
require.Equal(t, uint32(42), v)
v, err = readUint32(conn, 5*time.Second)
require.NoError(t, err)
require.Equal(t, uint32(math.MaxUint32), v)
v, err = readUint32(conn, 5*time.Second)
require.NoError(t, err)
require.Equal(t, uint32(1), v)
},
},
{
name: "triggers timeout on last read",
writeFn: func(conn net.Conn) {
_ = binary.Write(conn, binary.BigEndian, uint32(42))
_ = binary.Write(conn, binary.BigEndian, uint32(math.MaxUint32))
_ = binary.Write(conn, binary.BigEndian, uint16(1)) // half as many bytes as expected
},
readFn: func(t *testing.T, conn net.Conn) {
t.Helper()
v, err := readUint32(conn, 5*time.Second)
require.NoError(t, err)
require.Equal(t, uint32(42), v)
v, err = readUint32(conn, 5*time.Second)
require.NoError(t, err)
require.Equal(t, uint32(math.MaxUint32), v)
_, err = readUint32(conn, 50*time.Millisecond)
require.Error(t, err)
nerr, ok := err.(net.Error)
require.True(t, ok)
require.True(t, nerr.Timeout())
},
},
}
for _, tc := range cases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
var doneWg sync.WaitGroup
defer doneWg.Wait()
client, server := net.Pipe()
defer client.Close()
defer server.Close()
// Client pushes some data.
doneWg.Add(1)
go func() {
doneWg.Done()
tc.writeFn(client)
}()
// The server tests the function for us.
tc.readFn(t, server)
})
}
}