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.
This commit is contained in:
R.B. Boyer 2020-03-09 15:59:02 -05:00 committed by GitHub
parent da2639adf5
commit a7fb26f50f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
153 changed files with 9688 additions and 679 deletions

View File

@ -498,6 +498,7 @@ func (a *Agent) Start() error {
Datacenter: a.config.Datacenter,
Segment: a.config.SegmentName,
},
TLSConfigurator: a.tlsConfigurator,
})
if err != nil {
return err
@ -562,7 +563,9 @@ func (a *Agent) Start() error {
// start retry join
go a.retryJoinLAN()
go a.retryJoinWAN()
if a.config.ServerMode {
go a.retryJoinWAN()
}
return nil
}
@ -575,7 +578,7 @@ func (a *Agent) setupClientAutoEncrypt() (*structs.SignedResponse, error) {
if err != nil && len(addrs) == 0 {
return nil, err
}
addrs = append(addrs, retryJoinAddrs(disco, "LAN", a.config.RetryJoinLAN, a.logger)...)
addrs = append(addrs, retryJoinAddrs(disco, retryJoinSerfVariant, "LAN", a.config.RetryJoinLAN, a.logger)...)
reply, priv, err := client.RequestAutoEncryptCerts(addrs, a.config.ServerPort, a.tokens.AgentToken(), a.InterruptStartCh)
if err != nil {
@ -1341,6 +1344,7 @@ func (a *Agent) consulConfig() (*consul.Config, error) {
// Copy the Connect CA bootstrap config
if a.config.ConnectEnabled {
base.ConnectEnabled = true
base.ConnectMeshGatewayWANFederationEnabled = a.config.ConnectMeshGatewayWANFederationEnabled
// Allow config to specify cluster_id provided it's a valid UUID. This is
// meant only for tests where a deterministic ID makes fixtures much simpler
@ -1899,6 +1903,34 @@ func (a *Agent) JoinWAN(addrs []string) (n int, err error) {
return
}
// PrimaryMeshGatewayAddressesReadyCh returns a channel that will be closed
// when federation state replication ships back at least one primary mesh
// gateway (not via fallback config).
func (a *Agent) PrimaryMeshGatewayAddressesReadyCh() <-chan struct{} {
if srv, ok := a.delegate.(*consul.Server); ok {
return srv.PrimaryMeshGatewayAddressesReadyCh()
}
return nil
}
// PickRandomMeshGatewaySuitableForDialing is a convenience function used for writing tests.
func (a *Agent) PickRandomMeshGatewaySuitableForDialing(dc string) string {
if srv, ok := a.delegate.(*consul.Server); ok {
return srv.PickRandomMeshGatewaySuitableForDialing(dc)
}
return ""
}
// RefreshPrimaryGatewayFallbackAddresses is used to update the list of current
// fallback addresses for locating mesh gateways in the primary datacenter.
func (a *Agent) RefreshPrimaryGatewayFallbackAddresses(addrs []string) error {
if srv, ok := a.delegate.(*consul.Server); ok {
srv.RefreshPrimaryGatewayFallbackAddresses(addrs)
return nil
}
return fmt.Errorf("Must be a server to track mesh gateways in the primary datacenter")
}
// ForceLeave is used to remove a failed node from the cluster
func (a *Agent) ForceLeave(node string, prune bool) (err error) {
a.logger.Info("Force leaving node", "node", node)
@ -4265,6 +4297,14 @@ func (a *Agent) registerCache() {
RefreshTimer: 0 * time.Second,
RefreshTimeout: 10 * time.Minute,
})
a.cache.RegisterType(cachetype.FederationStateListMeshGatewaysName, &cachetype.FederationStateListMeshGateways{
RPC: a,
}, &cache.RegisterOptions{
Refresh: true,
RefreshTimer: 0 * time.Second,
RefreshTimeout: 10 * time.Minute,
})
}
func (a *Agent) LocalState() *local.State {

View File

@ -461,7 +461,11 @@ func (s *HTTPServer) AgentJoin(resp http.ResponseWriter, req *http.Request) (int
// Get the address
addr := strings.TrimPrefix(req.URL.Path, "/v1/agent/join/")
if wan {
if s.agent.config.ConnectMeshGatewayWANFederationEnabled {
return nil, fmt.Errorf("WAN join is disabled when wan federation via mesh gateways is enabled")
}
_, err = s.agent.JoinWAN([]string{addr})
} else {
_, err = s.agent.JoinLAN([]string{addr})
@ -904,7 +908,7 @@ func (s *HTTPServer) AgentRegisterService(resp http.ResponseWriter, req *http.Re
return nil, nil
}
}
if err := structs.ValidateMetadata(ns.Meta, false); err != nil {
if err := structs.ValidateServiceMetadata(ns.Kind, ns.Meta, false); err != nil {
resp.WriteHeader(http.StatusBadRequest)
fmt.Fprint(resp, fmt.Errorf("Invalid Service Meta: %v", err))
return nil, nil

View File

@ -17,8 +17,7 @@ import (
"testing"
"time"
"github.com/hashicorp/consul/testrpc"
"github.com/google/tcpproxy"
"github.com/hashicorp/consul/agent/cache"
cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/hashicorp/consul/agent/checks"
@ -26,11 +25,14 @@ import (
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/ipaddr"
"github.com/hashicorp/consul/sdk/freeport"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-uuid"
"github.com/hashicorp/serf/serf"
"github.com/pascaldekloe/goe/verify"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
@ -4150,3 +4152,273 @@ LOOP:
}
}
}
// This is a mirror of a similar test in agent/consul/server_test.go
func TestAgent_JoinWAN_viaMeshGateway(t *testing.T) {
t.Parallel()
gwPort := freeport.MustTake(1)
defer freeport.Return(gwPort)
gwAddr := ipaddr.FormatAddressPort("127.0.0.1", gwPort[0])
// Due to some ordering, we'll have to manually configure these ports in
// advance.
secondaryRPCPorts := freeport.MustTake(2)
defer freeport.Return(secondaryRPCPorts)
a1 := NewTestAgent(t, t.Name()+"-bob", `
domain = "consul"
node_name = "bob"
datacenter = "dc1"
primary_datacenter = "dc1"
# tls
ca_file = "../test/hostname/CertAuth.crt"
cert_file = "../test/hostname/Bob.crt"
key_file = "../test/hostname/Bob.key"
verify_incoming = true
verify_outgoing = true
verify_server_hostname = true
# wanfed
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`)
defer a1.Shutdown()
testrpc.WaitForTestAgent(t, a1.RPC, "dc1")
// We'll use the same gateway for all datacenters since it doesn't care.
var (
rpcAddr1 = ipaddr.FormatAddressPort("127.0.0.1", a1.Config.ServerPort)
rpcAddr2 = ipaddr.FormatAddressPort("127.0.0.1", secondaryRPCPorts[0])
rpcAddr3 = ipaddr.FormatAddressPort("127.0.0.1", secondaryRPCPorts[1])
)
var p tcpproxy.Proxy
p.AddSNIRoute(gwAddr, "bob.server.dc1.consul", tcpproxy.To(rpcAddr1))
p.AddSNIRoute(gwAddr, "server.dc1.consul", tcpproxy.To(rpcAddr1))
p.AddSNIRoute(gwAddr, "betty.server.dc2.consul", tcpproxy.To(rpcAddr2))
p.AddSNIRoute(gwAddr, "server.dc2.consul", tcpproxy.To(rpcAddr2))
p.AddSNIRoute(gwAddr, "bonnie.server.dc3.consul", tcpproxy.To(rpcAddr3))
p.AddSNIRoute(gwAddr, "server.dc3.consul", tcpproxy.To(rpcAddr3))
p.AddStopACMESearch(gwAddr)
require.NoError(t, p.Start())
defer func() {
p.Close()
p.Wait()
}()
t.Logf("routing %s => %s", "{bob.,}server.dc1.consul", rpcAddr1)
t.Logf("routing %s => %s", "{betty.,}server.dc2.consul", rpcAddr2)
t.Logf("routing %s => %s", "{bonnie.,}server.dc3.consul", rpcAddr3)
// Register this into the agent in dc1.
{
args := &structs.ServiceDefinition{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Name: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
}
req, err := http.NewRequest("PUT", "/v1/agent/service/register", jsonReader(args))
require.NoError(t, err)
obj, err := a1.srv.AgentRegisterService(nil, req)
require.NoError(t, err)
require.Nil(t, obj)
}
waitForFederationState := func(t *testing.T, a *TestAgent, dc string) {
retry.Run(t, func(r *retry.R) {
req, err := http.NewRequest("GET", "/v1/internal/federation-state/"+dc, nil)
require.NoError(r, err)
resp := httptest.NewRecorder()
obj, err := a.srv.FederationStateGet(resp, req)
require.NoError(r, err)
require.NotNil(r, obj)
out, ok := obj.(structs.FederationStateResponse)
require.True(r, ok)
require.NotNil(r, out.State)
require.Len(r, out.State.MeshGateways, 1)
})
}
// Wait until at least catalog AE and federation state AE fire.
waitForFederationState(t, a1, "dc1")
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, a1.PickRandomMeshGatewaySuitableForDialing("dc1"))
})
a2 := NewTestAgent(t, t.Name()+"-betty", `
domain = "consul"
node_name = "betty"
datacenter = "dc2"
primary_datacenter = "dc1"
# tls
ca_file = "../test/hostname/CertAuth.crt"
cert_file = "../test/hostname/Betty.crt"
key_file = "../test/hostname/Betty.key"
verify_incoming = true
verify_outgoing = true
verify_server_hostname = true
ports {
server = `+strconv.Itoa(secondaryRPCPorts[0])+`
}
# wanfed
primary_gateways = ["`+gwAddr+`"]
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`)
defer a2.Shutdown()
testrpc.WaitForTestAgent(t, a2.RPC, "dc2")
a3 := NewTestAgent(t, t.Name()+"-bonnie", `
domain = "consul"
node_name = "bonnie"
datacenter = "dc3"
primary_datacenter = "dc1"
# tls
ca_file = "../test/hostname/CertAuth.crt"
cert_file = "../test/hostname/Bonnie.crt"
key_file = "../test/hostname/Bonnie.key"
verify_incoming = true
verify_outgoing = true
verify_server_hostname = true
ports {
server = `+strconv.Itoa(secondaryRPCPorts[1])+`
}
# wanfed
primary_gateways = ["`+gwAddr+`"]
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`)
defer a3.Shutdown()
testrpc.WaitForTestAgent(t, a3.RPC, "dc3")
// The primary_gateways config setting should cause automatic mesh join.
// Assert that the secondaries have joined the primary.
findPrimary := func(r *retry.R, a *TestAgent) *serf.Member {
var primary *serf.Member
for _, m := range a.WANMembers() {
if m.Tags["dc"] == "dc1" {
require.Nil(r, primary, "already found one node in dc1")
primary = &m
}
}
require.NotNil(r, primary)
return primary
}
retry.Run(t, func(r *retry.R) {
p2, p3 := findPrimary(r, a2), findPrimary(r, a3)
require.Equal(r, "bob.dc1", p2.Name)
require.Equal(r, "bob.dc1", p3.Name)
})
testrpc.WaitForLeader(t, a2.RPC, "dc2")
testrpc.WaitForLeader(t, a3.RPC, "dc3")
// Now we can register this into the catalog in dc2 and dc3.
{
args := &structs.ServiceDefinition{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Name: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
}
req, err := http.NewRequest("PUT", "/v1/agent/service/register", jsonReader(args))
require.NoError(t, err)
obj, err := a2.srv.AgentRegisterService(nil, req)
require.NoError(t, err)
require.Nil(t, obj)
}
{
args := &structs.ServiceDefinition{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Name: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
}
req, err := http.NewRequest("PUT", "/v1/agent/service/register", jsonReader(args))
require.NoError(t, err)
obj, err := a3.srv.AgentRegisterService(nil, req)
require.NoError(t, err)
require.Nil(t, obj)
}
// Wait until federation state replication functions
waitForFederationState(t, a1, "dc1")
waitForFederationState(t, a1, "dc2")
waitForFederationState(t, a1, "dc3")
waitForFederationState(t, a2, "dc1")
waitForFederationState(t, a2, "dc2")
waitForFederationState(t, a2, "dc3")
waitForFederationState(t, a3, "dc1")
waitForFederationState(t, a3, "dc2")
waitForFederationState(t, a3, "dc3")
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, a1.PickRandomMeshGatewaySuitableForDialing("dc1"))
require.NotEmpty(r, a1.PickRandomMeshGatewaySuitableForDialing("dc2"))
require.NotEmpty(r, a1.PickRandomMeshGatewaySuitableForDialing("dc3"))
require.NotEmpty(r, a2.PickRandomMeshGatewaySuitableForDialing("dc1"))
require.NotEmpty(r, a2.PickRandomMeshGatewaySuitableForDialing("dc2"))
require.NotEmpty(r, a2.PickRandomMeshGatewaySuitableForDialing("dc3"))
require.NotEmpty(r, a3.PickRandomMeshGatewaySuitableForDialing("dc1"))
require.NotEmpty(r, a3.PickRandomMeshGatewaySuitableForDialing("dc2"))
require.NotEmpty(r, a3.PickRandomMeshGatewaySuitableForDialing("dc3"))
})
retry.Run(t, func(r *retry.R) {
if got, want := len(a1.WANMembers()), 3; got != want {
r.Fatalf("got %d WAN members want at least %d", got, want)
}
if got, want := len(a2.WANMembers()), 3; got != want {
r.Fatalf("got %d WAN members want at least %d", got, want)
}
if got, want := len(a3.WANMembers()), 3; got != want {
r.Fatalf("got %d WAN members want at least %d", got, want)
}
})
// Ensure we can do some trivial RPC in all directions.
agents := map[string]*TestAgent{"dc1": a1, "dc2": a2, "dc3": a3}
names := map[string]string{"dc1": "bob", "dc2": "betty", "dc3": "bonnie"}
for _, srcDC := range []string{"dc1", "dc2", "dc3"} {
a := agents[srcDC]
for _, dstDC := range []string{"dc1", "dc2", "dc3"} {
if srcDC == dstDC {
continue
}
t.Run(srcDC+" to "+dstDC, func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/catalog/nodes?dc="+dstDC, nil)
require.NoError(t, err)
resp := httptest.NewRecorder()
obj, err := a.srv.CatalogNodes(resp, req)
require.NoError(t, err)
require.NotNil(t, obj)
nodes, ok := obj.(structs.Nodes)
require.True(t, ok)
require.Len(t, nodes, 1)
node := nodes[0]
require.Equal(t, dstDC, node.Datacenter)
require.Equal(t, names[dstDC], node.Node)
})
}
}
}

View File

@ -0,0 +1,55 @@
package cachetype
import (
"fmt"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/agent/structs"
)
// Recommended name for registration.
const FederationStateListMeshGatewaysName = "federation-state-list-mesh-gateways"
// FederationState supports fetching federation states.
type FederationStateListMeshGateways struct {
RPC RPC
}
func (c *FederationStateListMeshGateways) Fetch(opts cache.FetchOptions, req cache.Request) (cache.FetchResult, error) {
var result cache.FetchResult
// The request should be a DCSpecificRequest.
reqReal, ok := req.(*structs.DCSpecificRequest)
if !ok {
return result, fmt.Errorf(
"Internal cache failure: request wrong type: %T", req)
}
// Lightweight copy this object so that manipulating QueryOptions doesn't race.
dup := *reqReal
reqReal = &dup
// Set the minimum query index to our current index so we block
reqReal.QueryOptions.MinQueryIndex = opts.MinIndex
reqReal.QueryOptions.MaxQueryTime = opts.Timeout
// Always allow stale - there's no point in hitting leader if the request is
// going to be served from cache and end up arbitrarily stale anyway. This
// allows cached service-discover to automatically read scale across all
// servers too.
reqReal.AllowStale = true
// Fetch
var reply structs.DatacenterIndexedCheckServiceNodes
if err := c.RPC.RPC("FederationState.ListMeshGateways", reqReal, &reply); err != nil {
return result, err
}
result.Value = &reply
result.Index = reply.QueryMeta.Index
return result, nil
}
func (c *FederationStateListMeshGateways) SupportsBlocking() bool {
return true
}

View File

@ -0,0 +1,107 @@
package cachetype
import (
"testing"
"time"
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
)
func TestFederationStateListMeshGateways(t *testing.T) {
rpc := TestRPC(t)
typ := &FederationStateListMeshGateways{RPC: rpc}
// Expect the proper RPC call. This also sets the expected value
// since that is return-by-pointer in the arguments.
var resp *structs.DatacenterIndexedCheckServiceNodes
rpc.On("RPC", "FederationState.ListMeshGateways", mock.Anything, mock.Anything).Return(nil).
Run(func(args mock.Arguments) {
req := args.Get(1).(*structs.DCSpecificRequest)
require.Equal(t, uint64(24), req.QueryOptions.MinQueryIndex)
require.Equal(t, 1*time.Second, req.QueryOptions.MaxQueryTime)
require.True(t, req.AllowStale)
reply := args.Get(2).(*structs.DatacenterIndexedCheckServiceNodes)
reply.DatacenterNodes = map[string]structs.CheckServiceNodes{
"dc9": []structs.CheckServiceNode{
{
Node: &structs.Node{
ID: "664bac9f-4de7-4f1b-ad35-0e5365e8f329",
Node: "gateway1",
Datacenter: "dc9",
Address: "1.2.3.4",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 1111,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
{
Node: &structs.Node{
ID: "3fb9a696-8209-4eee-a1f7-48600deb9716",
Node: "gateway2",
Datacenter: "dc9",
Address: "9.8.7.6",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 2222,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
},
}
reply.QueryMeta.Index = 48
resp = reply
})
// Fetch
resultA, err := typ.Fetch(cache.FetchOptions{
MinIndex: 24,
Timeout: 1 * time.Second,
}, &structs.DCSpecificRequest{
Datacenter: "dc1",
})
require.NoError(t, err)
require.Equal(t, cache.FetchResult{
Value: resp,
Index: 48,
}, resultA)
rpc.AssertExpectations(t)
}
func TestFederationStateListMeshGateways_badReqType(t *testing.T) {
rpc := TestRPC(t)
typ := &FederationStateListMeshGateways{RPC: rpc}
// Fetch
_, err := typ.Fetch(cache.FetchOptions{}, cache.TestRequest(
t, cache.RequestInfo{Key: "foo", MinIndex: 64}))
require.Error(t, err)
require.Contains(t, err.Error(), "wrong type")
rpc.AssertExpectations(t)
}

View File

@ -618,6 +618,10 @@ func (b *Builder) Build() (rt RuntimeConfig, err error) {
connectEnabled := b.boolVal(c.Connect.Enabled)
connectCAProvider := b.stringVal(c.Connect.CAProvider)
connectCAConfig := c.Connect.CAConfig
connectMeshGatewayWANFederationEnabled := b.boolVal(c.Connect.MeshGatewayWANFederationEnabled)
if connectMeshGatewayWANFederationEnabled && !connectEnabled {
return RuntimeConfig{}, fmt.Errorf("'connect.enable_mesh_gateway_wan_federation=true' requires 'connect.enabled=true'")
}
if connectCAConfig != nil {
lib.TranslateKeys(connectCAConfig, map[string]string{
// Consul CA config
@ -857,129 +861,132 @@ func (b *Builder) Build() (rt RuntimeConfig, err error) {
},
// Agent
AdvertiseAddrLAN: advertiseAddrLAN,
AdvertiseAddrWAN: advertiseAddrWAN,
BindAddr: bindAddr,
Bootstrap: b.boolVal(c.Bootstrap),
BootstrapExpect: b.intVal(c.BootstrapExpect),
CAFile: b.stringVal(c.CAFile),
CAPath: b.stringVal(c.CAPath),
CertFile: b.stringVal(c.CertFile),
CheckUpdateInterval: b.durationVal("check_update_interval", c.CheckUpdateInterval),
CheckOutputMaxSize: b.intValWithDefault(c.CheckOutputMaxSize, 4096),
Checks: checks,
ClientAddrs: clientAddrs,
ConfigEntryBootstrap: configEntries,
AutoEncryptTLS: autoEncryptTLS,
AutoEncryptDNSSAN: autoEncryptDNSSAN,
AutoEncryptIPSAN: autoEncryptIPSAN,
AutoEncryptAllowTLS: autoEncryptAllowTLS,
ConnectEnabled: connectEnabled,
ConnectCAProvider: connectCAProvider,
ConnectCAConfig: connectCAConfig,
ConnectSidecarMinPort: sidecarMinPort,
ConnectSidecarMaxPort: sidecarMaxPort,
ExposeMinPort: exposeMinPort,
ExposeMaxPort: exposeMaxPort,
DataDir: b.stringVal(c.DataDir),
Datacenter: datacenter,
DefaultQueryTime: b.durationVal("default_query_time", c.DefaultQueryTime),
DevMode: b.boolVal(b.Flags.DevMode),
DisableAnonymousSignature: b.boolVal(c.DisableAnonymousSignature),
DisableCoordinates: b.boolVal(c.DisableCoordinates),
DisableHostNodeID: b.boolVal(c.DisableHostNodeID),
DisableHTTPUnprintableCharFilter: b.boolVal(c.DisableHTTPUnprintableCharFilter),
DisableKeyringFile: b.boolVal(c.DisableKeyringFile),
DisableRemoteExec: b.boolVal(c.DisableRemoteExec),
DisableUpdateCheck: b.boolVal(c.DisableUpdateCheck),
DiscardCheckOutput: b.boolVal(c.DiscardCheckOutput),
DiscoveryMaxStale: b.durationVal("discovery_max_stale", c.DiscoveryMaxStale),
EnableAgentTLSForChecks: b.boolVal(c.EnableAgentTLSForChecks),
EnableCentralServiceConfig: b.boolVal(c.EnableCentralServiceConfig),
EnableDebug: b.boolVal(c.EnableDebug),
EnableRemoteScriptChecks: enableRemoteScriptChecks,
EnableLocalScriptChecks: enableLocalScriptChecks,
EnableSyslog: b.boolVal(c.EnableSyslog),
EnableUI: b.boolVal(c.UI),
EncryptKey: b.stringVal(c.EncryptKey),
EncryptVerifyIncoming: b.boolVal(c.EncryptVerifyIncoming),
EncryptVerifyOutgoing: b.boolVal(c.EncryptVerifyOutgoing),
GRPCPort: grpcPort,
GRPCAddrs: grpcAddrs,
HTTPMaxConnsPerClient: b.intVal(c.Limits.HTTPMaxConnsPerClient),
HTTPSHandshakeTimeout: b.durationVal("limits.https_handshake_timeout", c.Limits.HTTPSHandshakeTimeout),
KeyFile: b.stringVal(c.KeyFile),
KVMaxValueSize: b.uint64Val(c.Limits.KVMaxValueSize),
LeaveDrainTime: b.durationVal("performance.leave_drain_time", c.Performance.LeaveDrainTime),
LeaveOnTerm: leaveOnTerm,
LogLevel: b.stringVal(c.LogLevel),
LogJSON: b.boolVal(c.LogJSON),
LogFile: b.stringVal(c.LogFile),
LogRotateBytes: b.intVal(c.LogRotateBytes),
LogRotateDuration: b.durationVal("log_rotate_duration", c.LogRotateDuration),
LogRotateMaxFiles: b.intVal(c.LogRotateMaxFiles),
MaxQueryTime: b.durationVal("max_query_time", c.MaxQueryTime),
NodeID: types.NodeID(b.stringVal(c.NodeID)),
NodeMeta: c.NodeMeta,
NodeName: b.nodeName(c.NodeName),
NonVotingServer: b.boolVal(c.NonVotingServer),
PidFile: b.stringVal(c.PidFile),
PrimaryDatacenter: primaryDatacenter,
RPCAdvertiseAddr: rpcAdvertiseAddr,
RPCBindAddr: rpcBindAddr,
RPCHandshakeTimeout: b.durationVal("limits.rpc_handshake_timeout", c.Limits.RPCHandshakeTimeout),
RPCHoldTimeout: b.durationVal("performance.rpc_hold_timeout", c.Performance.RPCHoldTimeout),
RPCMaxBurst: b.intVal(c.Limits.RPCMaxBurst),
RPCMaxConnsPerClient: b.intVal(c.Limits.RPCMaxConnsPerClient),
RPCProtocol: b.intVal(c.RPCProtocol),
RPCRateLimit: rate.Limit(b.float64Val(c.Limits.RPCRate)),
RaftProtocol: b.intVal(c.RaftProtocol),
RaftSnapshotThreshold: b.intVal(c.RaftSnapshotThreshold),
RaftSnapshotInterval: b.durationVal("raft_snapshot_interval", c.RaftSnapshotInterval),
RaftTrailingLogs: b.intVal(c.RaftTrailingLogs),
ReconnectTimeoutLAN: b.durationVal("reconnect_timeout", c.ReconnectTimeoutLAN),
ReconnectTimeoutWAN: b.durationVal("reconnect_timeout_wan", c.ReconnectTimeoutWAN),
RejoinAfterLeave: b.boolVal(c.RejoinAfterLeave),
RetryJoinIntervalLAN: b.durationVal("retry_interval", c.RetryJoinIntervalLAN),
RetryJoinIntervalWAN: b.durationVal("retry_interval_wan", c.RetryJoinIntervalWAN),
RetryJoinLAN: b.expandAllOptionalAddrs("retry_join", c.RetryJoinLAN),
RetryJoinMaxAttemptsLAN: b.intVal(c.RetryJoinMaxAttemptsLAN),
RetryJoinMaxAttemptsWAN: b.intVal(c.RetryJoinMaxAttemptsWAN),
RetryJoinWAN: b.expandAllOptionalAddrs("retry_join_wan", c.RetryJoinWAN),
SegmentName: b.stringVal(c.SegmentName),
Segments: segments,
SerfAdvertiseAddrLAN: serfAdvertiseAddrLAN,
SerfAdvertiseAddrWAN: serfAdvertiseAddrWAN,
SerfBindAddrLAN: serfBindAddrLAN,
SerfBindAddrWAN: serfBindAddrWAN,
SerfPortLAN: serfPortLAN,
SerfPortWAN: serfPortWAN,
ServerMode: b.boolVal(c.ServerMode),
ServerName: b.stringVal(c.ServerName),
ServerPort: serverPort,
Services: services,
SessionTTLMin: b.durationVal("session_ttl_min", c.SessionTTLMin),
SkipLeaveOnInt: skipLeaveOnInt,
StartJoinAddrsLAN: b.expandAllOptionalAddrs("start_join", c.StartJoinAddrsLAN),
StartJoinAddrsWAN: b.expandAllOptionalAddrs("start_join_wan", c.StartJoinAddrsWAN),
SyslogFacility: b.stringVal(c.SyslogFacility),
TLSCipherSuites: b.tlsCipherSuites("tls_cipher_suites", c.TLSCipherSuites),
TLSMinVersion: b.stringVal(c.TLSMinVersion),
TLSPreferServerCipherSuites: b.boolVal(c.TLSPreferServerCipherSuites),
TaggedAddresses: c.TaggedAddresses,
TranslateWANAddrs: b.boolVal(c.TranslateWANAddrs),
TxnMaxReqLen: b.uint64Val(c.Limits.TxnMaxReqLen),
UIDir: b.stringVal(c.UIDir),
UIContentPath: UIPathBuilder(b.stringVal(c.UIContentPath)),
UnixSocketGroup: b.stringVal(c.UnixSocket.Group),
UnixSocketMode: b.stringVal(c.UnixSocket.Mode),
UnixSocketUser: b.stringVal(c.UnixSocket.User),
VerifyIncoming: b.boolVal(c.VerifyIncoming),
VerifyIncomingHTTPS: b.boolVal(c.VerifyIncomingHTTPS),
VerifyIncomingRPC: b.boolVal(c.VerifyIncomingRPC),
VerifyOutgoing: verifyOutgoing,
VerifyServerHostname: verifyServerName,
Watches: c.Watches,
AdvertiseAddrLAN: advertiseAddrLAN,
AdvertiseAddrWAN: advertiseAddrWAN,
BindAddr: bindAddr,
Bootstrap: b.boolVal(c.Bootstrap),
BootstrapExpect: b.intVal(c.BootstrapExpect),
CAFile: b.stringVal(c.CAFile),
CAPath: b.stringVal(c.CAPath),
CertFile: b.stringVal(c.CertFile),
CheckUpdateInterval: b.durationVal("check_update_interval", c.CheckUpdateInterval),
CheckOutputMaxSize: b.intValWithDefault(c.CheckOutputMaxSize, 4096),
Checks: checks,
ClientAddrs: clientAddrs,
ConfigEntryBootstrap: configEntries,
AutoEncryptTLS: autoEncryptTLS,
AutoEncryptDNSSAN: autoEncryptDNSSAN,
AutoEncryptIPSAN: autoEncryptIPSAN,
AutoEncryptAllowTLS: autoEncryptAllowTLS,
ConnectEnabled: connectEnabled,
ConnectCAProvider: connectCAProvider,
ConnectCAConfig: connectCAConfig,
ConnectMeshGatewayWANFederationEnabled: connectMeshGatewayWANFederationEnabled,
ConnectSidecarMinPort: sidecarMinPort,
ConnectSidecarMaxPort: sidecarMaxPort,
ExposeMinPort: exposeMinPort,
ExposeMaxPort: exposeMaxPort,
DataDir: b.stringVal(c.DataDir),
Datacenter: datacenter,
DefaultQueryTime: b.durationVal("default_query_time", c.DefaultQueryTime),
DevMode: b.boolVal(b.Flags.DevMode),
DisableAnonymousSignature: b.boolVal(c.DisableAnonymousSignature),
DisableCoordinates: b.boolVal(c.DisableCoordinates),
DisableHostNodeID: b.boolVal(c.DisableHostNodeID),
DisableHTTPUnprintableCharFilter: b.boolVal(c.DisableHTTPUnprintableCharFilter),
DisableKeyringFile: b.boolVal(c.DisableKeyringFile),
DisableRemoteExec: b.boolVal(c.DisableRemoteExec),
DisableUpdateCheck: b.boolVal(c.DisableUpdateCheck),
DiscardCheckOutput: b.boolVal(c.DiscardCheckOutput),
DiscoveryMaxStale: b.durationVal("discovery_max_stale", c.DiscoveryMaxStale),
EnableAgentTLSForChecks: b.boolVal(c.EnableAgentTLSForChecks),
EnableCentralServiceConfig: b.boolVal(c.EnableCentralServiceConfig),
EnableDebug: b.boolVal(c.EnableDebug),
EnableRemoteScriptChecks: enableRemoteScriptChecks,
EnableLocalScriptChecks: enableLocalScriptChecks,
EnableSyslog: b.boolVal(c.EnableSyslog),
EnableUI: b.boolVal(c.UI),
EncryptKey: b.stringVal(c.EncryptKey),
EncryptVerifyIncoming: b.boolVal(c.EncryptVerifyIncoming),
EncryptVerifyOutgoing: b.boolVal(c.EncryptVerifyOutgoing),
GRPCPort: grpcPort,
GRPCAddrs: grpcAddrs,
HTTPMaxConnsPerClient: b.intVal(c.Limits.HTTPMaxConnsPerClient),
HTTPSHandshakeTimeout: b.durationVal("limits.https_handshake_timeout", c.Limits.HTTPSHandshakeTimeout),
KeyFile: b.stringVal(c.KeyFile),
KVMaxValueSize: b.uint64Val(c.Limits.KVMaxValueSize),
LeaveDrainTime: b.durationVal("performance.leave_drain_time", c.Performance.LeaveDrainTime),
LeaveOnTerm: leaveOnTerm,
LogLevel: b.stringVal(c.LogLevel),
LogJSON: b.boolVal(c.LogJSON),
LogFile: b.stringVal(c.LogFile),
LogRotateBytes: b.intVal(c.LogRotateBytes),
LogRotateDuration: b.durationVal("log_rotate_duration", c.LogRotateDuration),
LogRotateMaxFiles: b.intVal(c.LogRotateMaxFiles),
MaxQueryTime: b.durationVal("max_query_time", c.MaxQueryTime),
NodeID: types.NodeID(b.stringVal(c.NodeID)),
NodeMeta: c.NodeMeta,
NodeName: b.nodeName(c.NodeName),
NonVotingServer: b.boolVal(c.NonVotingServer),
PidFile: b.stringVal(c.PidFile),
PrimaryDatacenter: primaryDatacenter,
PrimaryGateways: b.expandAllOptionalAddrs("primary_gateways", c.PrimaryGateways),
PrimaryGatewaysInterval: b.durationVal("primary_gateways_interval", c.PrimaryGatewaysInterval),
RPCAdvertiseAddr: rpcAdvertiseAddr,
RPCBindAddr: rpcBindAddr,
RPCHandshakeTimeout: b.durationVal("limits.rpc_handshake_timeout", c.Limits.RPCHandshakeTimeout),
RPCHoldTimeout: b.durationVal("performance.rpc_hold_timeout", c.Performance.RPCHoldTimeout),
RPCMaxBurst: b.intVal(c.Limits.RPCMaxBurst),
RPCMaxConnsPerClient: b.intVal(c.Limits.RPCMaxConnsPerClient),
RPCProtocol: b.intVal(c.RPCProtocol),
RPCRateLimit: rate.Limit(b.float64Val(c.Limits.RPCRate)),
RaftProtocol: b.intVal(c.RaftProtocol),
RaftSnapshotThreshold: b.intVal(c.RaftSnapshotThreshold),
RaftSnapshotInterval: b.durationVal("raft_snapshot_interval", c.RaftSnapshotInterval),
RaftTrailingLogs: b.intVal(c.RaftTrailingLogs),
ReconnectTimeoutLAN: b.durationVal("reconnect_timeout", c.ReconnectTimeoutLAN),
ReconnectTimeoutWAN: b.durationVal("reconnect_timeout_wan", c.ReconnectTimeoutWAN),
RejoinAfterLeave: b.boolVal(c.RejoinAfterLeave),
RetryJoinIntervalLAN: b.durationVal("retry_interval", c.RetryJoinIntervalLAN),
RetryJoinIntervalWAN: b.durationVal("retry_interval_wan", c.RetryJoinIntervalWAN),
RetryJoinLAN: b.expandAllOptionalAddrs("retry_join", c.RetryJoinLAN),
RetryJoinMaxAttemptsLAN: b.intVal(c.RetryJoinMaxAttemptsLAN),
RetryJoinMaxAttemptsWAN: b.intVal(c.RetryJoinMaxAttemptsWAN),
RetryJoinWAN: b.expandAllOptionalAddrs("retry_join_wan", c.RetryJoinWAN),
SegmentName: b.stringVal(c.SegmentName),
Segments: segments,
SerfAdvertiseAddrLAN: serfAdvertiseAddrLAN,
SerfAdvertiseAddrWAN: serfAdvertiseAddrWAN,
SerfBindAddrLAN: serfBindAddrLAN,
SerfBindAddrWAN: serfBindAddrWAN,
SerfPortLAN: serfPortLAN,
SerfPortWAN: serfPortWAN,
ServerMode: b.boolVal(c.ServerMode),
ServerName: b.stringVal(c.ServerName),
ServerPort: serverPort,
Services: services,
SessionTTLMin: b.durationVal("session_ttl_min", c.SessionTTLMin),
SkipLeaveOnInt: skipLeaveOnInt,
StartJoinAddrsLAN: b.expandAllOptionalAddrs("start_join", c.StartJoinAddrsLAN),
StartJoinAddrsWAN: b.expandAllOptionalAddrs("start_join_wan", c.StartJoinAddrsWAN),
SyslogFacility: b.stringVal(c.SyslogFacility),
TLSCipherSuites: b.tlsCipherSuites("tls_cipher_suites", c.TLSCipherSuites),
TLSMinVersion: b.stringVal(c.TLSMinVersion),
TLSPreferServerCipherSuites: b.boolVal(c.TLSPreferServerCipherSuites),
TaggedAddresses: c.TaggedAddresses,
TranslateWANAddrs: b.boolVal(c.TranslateWANAddrs),
TxnMaxReqLen: b.uint64Val(c.Limits.TxnMaxReqLen),
UIDir: b.stringVal(c.UIDir),
UIContentPath: UIPathBuilder(b.stringVal(c.UIContentPath)),
UnixSocketGroup: b.stringVal(c.UnixSocket.Group),
UnixSocketMode: b.stringVal(c.UnixSocket.Mode),
UnixSocketUser: b.stringVal(c.UnixSocket.User),
VerifyIncoming: b.boolVal(c.VerifyIncoming),
VerifyIncomingHTTPS: b.boolVal(c.VerifyIncomingHTTPS),
VerifyIncomingRPC: b.boolVal(c.VerifyIncomingRPC),
VerifyOutgoing: verifyOutgoing,
VerifyServerHostname: verifyServerName,
Watches: c.Watches,
}
if entCfg, err := b.BuildEnterpriseRuntimeConfig(&c); err != nil {
@ -1100,7 +1107,7 @@ func (b *Builder) Validate(rt RuntimeConfig) error {
if rt.DNSARecordLimit < 0 {
return fmt.Errorf("dns_config.a_record_limit cannot be %d. Must be greater than or equal to zero", rt.DNSARecordLimit)
}
if err := structs.ValidateMetadata(rt.NodeMeta, false); err != nil {
if err := structs.ValidateNodeMetadata(rt.NodeMeta, false); err != nil {
return fmt.Errorf("node_meta invalid: %v", err)
}
if rt.EncryptKey != "" {
@ -1109,6 +1116,29 @@ func (b *Builder) Validate(rt RuntimeConfig) error {
}
}
if rt.ConnectMeshGatewayWANFederationEnabled && !rt.ServerMode {
return fmt.Errorf("'connect.enable_mesh_gateway_wan_federation = true' requires 'server = true'")
}
if rt.ConnectMeshGatewayWANFederationEnabled && strings.ContainsAny(rt.NodeName, "/") {
return fmt.Errorf("'connect.enable_mesh_gateway_wan_federation = true' requires that 'node_name' not contain '/' characters")
}
if rt.ConnectMeshGatewayWANFederationEnabled {
if len(rt.StartJoinAddrsWAN) > 0 {
return fmt.Errorf("'start_join_wan' is incompatible with 'connect.enable_mesh_gateway_wan_federation = true'")
}
if len(rt.RetryJoinWAN) > 0 {
return fmt.Errorf("'retry_join_wan' is incompatible with 'connect.enable_mesh_gateway_wan_federation = true'")
}
}
if len(rt.PrimaryGateways) > 0 {
if !rt.ServerMode {
return fmt.Errorf("'primary_gateways' requires 'server = true'")
}
if rt.PrimaryDatacenter == rt.Datacenter {
return fmt.Errorf("'primary_gateways' should only be configured in a secondary datacenter")
}
}
// Check the data dir for signs of an un-migrated Consul 0.5.x or older
// server. Consul refuses to start if this is present to protect a server
// with existing data from starting on a fresh data set.
@ -1331,8 +1361,10 @@ func (b *Builder) serviceVal(v *ServiceDefinition) *structs.ServiceDefinition {
checks = append(checks, b.checkVal(v.Check).CheckType())
}
kind := b.serviceKindVal(v.Kind)
meta := make(map[string]string)
if err := structs.ValidateMetadata(v.Meta, false); err != nil {
if err := structs.ValidateServiceMetadata(kind, v.Meta, false); err != nil {
b.err = multierror.Append(fmt.Errorf("invalid meta for service %s: %v", b.stringVal(v.Name), err))
} else {
meta = v.Meta
@ -1351,7 +1383,7 @@ func (b *Builder) serviceVal(v *ServiceDefinition) *structs.ServiceDefinition {
b.err = multierror.Append(fmt.Errorf("Invalid weight definition for service %s: %s", b.stringVal(v.Name), err))
}
return &structs.ServiceDefinition{
Kind: b.serviceKindVal(v.Kind),
Kind: kind,
ID: b.stringVal(v.ID),
Name: b.stringVal(v.Name),
Tags: v.Tags,

View File

@ -250,6 +250,8 @@ type Config struct {
PidFile *string `json:"pid_file,omitempty" hcl:"pid_file" mapstructure:"pid_file"`
Ports Ports `json:"ports,omitempty" hcl:"ports" mapstructure:"ports"`
PrimaryDatacenter *string `json:"primary_datacenter,omitempty" hcl:"primary_datacenter" mapstructure:"primary_datacenter"`
PrimaryGateways []string `json:"primary_gateways" hcl:"primary_gateways" mapstructure:"primary_gateways"`
PrimaryGatewaysInterval *string `json:"primary_gateways_interval,omitempty" hcl:"primary_gateways_interval" mapstructure:"primary_gateways_interval"`
RPCProtocol *int `json:"protocol,omitempty" hcl:"protocol" mapstructure:"protocol"`
RaftProtocol *int `json:"raft_protocol,omitempty" hcl:"raft_protocol" mapstructure:"raft_protocol"`
RaftSnapshotThreshold *int `json:"raft_snapshot_threshold,omitempty" hcl:"raft_snapshot_threshold" mapstructure:"raft_snapshot_threshold"`
@ -586,9 +588,10 @@ type AutoEncrypt struct {
type Connect struct {
// Enabled opts the agent into connect. It should be set on all clients and
// servers in a cluster for correct connect operation.
Enabled *bool `json:"enabled,omitempty" hcl:"enabled" mapstructure:"enabled"`
CAProvider *string `json:"ca_provider,omitempty" hcl:"ca_provider" mapstructure:"ca_provider"`
CAConfig map[string]interface{} `json:"ca_config,omitempty" hcl:"ca_config" mapstructure:"ca_config"`
Enabled *bool `json:"enabled,omitempty" hcl:"enabled" mapstructure:"enabled"`
CAProvider *string `json:"ca_provider,omitempty" hcl:"ca_provider" mapstructure:"ca_provider"`
CAConfig map[string]interface{} `json:"ca_config,omitempty" hcl:"ca_config" mapstructure:"ca_config"`
MeshGatewayWANFederationEnabled *bool `json:"enable_mesh_gateway_wan_federation" hcl:"enable_mesh_gateway_wan_federation" mapstructure:"enable_mesh_gateway_wan_federation"`
}
// SOA is the configuration of SOA for DNS

View File

@ -64,6 +64,7 @@ func DefaultSource() Source {
encrypt_verify_outgoing = true
log_level = "INFO"
max_query_time = "600s"
primary_gateways_interval = "30s"
protocol = 2
retry_interval = "30s"
retry_interval_wan = "30s"

View File

@ -97,6 +97,7 @@ func AddFlags(fs *flag.FlagSet, f *Flags) {
add(&f.Config.RPCProtocol, "protocol", "Sets the protocol version. Defaults to latest.")
add(&f.Config.RaftProtocol, "raft-protocol", "Sets the Raft protocol version. Defaults to latest.")
add(&f.Config.DNSRecursors, "recursor", "Address of an upstream DNS server. Can be specified multiple times.")
add(&f.Config.PrimaryGateways, "primary-gateway", "Address of a mesh gateway in the primary datacenter to use to bootstrap WAN federation at start time with retries enabled. Can be specified multiple times.")
add(&f.Config.RejoinAfterLeave, "rejoin", "Ignores a previous leave and attempts to rejoin the cluster.")
add(&f.Config.RetryJoinIntervalLAN, "retry-interval", "Time to wait between join attempts.")
add(&f.Config.RetryJoinIntervalWAN, "retry-interval-wan", "Time to wait between join -wan attempts.")

View File

@ -84,6 +84,12 @@ func TestParseFlags(t *testing.T) {
args: []string{`-bootstrap`, `true`},
flags: Flags{Config: Config{Bootstrap: pBool(true)}, Args: []string{"true"}},
},
{
args: []string{`-primary-gateway`, `foo.local`, `-primary-gateway`, `bar.local`},
flags: Flags{Config: Config{PrimaryGateways: []string{
"foo.local", "bar.local",
}}},
},
}
for _, tt := range tests {

View File

@ -569,6 +569,10 @@ type RuntimeConfig struct {
// ConnectCAConfig is the config to use for the CA provider.
ConnectCAConfig map[string]interface{}
// ConnectMeshGatewayWANFederationEnabled determines if wan federation of
// datacenters should exclusively traverse mesh gateways.
ConnectMeshGatewayWANFederationEnabled bool
// ConnectTestCALeafRootChangeSpread is used to control how long the CA leaf
// cache with spread CSRs over when a root change occurs. For now we don't
// expose this in public config intentionally but could later with a rename.
@ -926,6 +930,22 @@ type RuntimeConfig struct {
// hcl: primary_datacenter = string
PrimaryDatacenter string
// PrimaryGateways is a list of addresses and/or go-discover expressions to
// discovery the mesh gateways in the primary datacenter. See
// https://www.consul.io/docs/agent/options.html#cloud-auto-joining for
// details.
//
// hcl: primary_gateways = []string
// flag: -primary-gateway string -primary-gateway string
PrimaryGateways []string
// PrimaryGatewaysInterval specifies the amount of time to wait in between discovery
// attempts on agent start. The minimum allowed value is 1 second and
// the default is 30s.
//
// hcl: primary_gateways_interval = "duration"
PrimaryGatewaysInterval time.Duration
// RPCAdvertiseAddr is the TCP address Consul advertises for its RPC endpoint.
// By default this is the bind address on the default RPC Server port. If the
// advertise address is specified then it is used.
@ -1041,14 +1061,14 @@ type RuntimeConfig struct {
// attempts on agent start. The minimum allowed value is 1 second and
// the default is 30s.
//
// hcl: retry_join = "duration"
// hcl: retry_interval = "duration"
RetryJoinIntervalLAN time.Duration
// RetryJoinIntervalWAN specifies the amount of time to wait in between join
// attempts on agent start. The minimum allowed value is 1 second and
// the default is 30s.
//
// hcl: retry_join_wan = "duration"
// hcl: retry_interval_wan = "duration"
RetryJoinIntervalWAN time.Duration
// RetryJoinLAN is a list of addresses and/or go-discover expressions to

View File

@ -621,6 +621,29 @@ func TestConfigFlagsAndEdgecases(t *testing.T) {
rt.DataDir = dataDir
},
},
{
desc: "-primary-gateway",
args: []string{
`-server`,
`-datacenter=dc2`,
`-primary-gateway=a`,
`-primary-gateway=b`,
`-data-dir=` + dataDir,
},
json: []string{`{ "primary_datacenter": "dc1" }`},
hcl: []string{`primary_datacenter = "dc1"`},
patch: func(rt *RuntimeConfig) {
rt.Datacenter = "dc2"
rt.PrimaryDatacenter = "dc1"
rt.ACLDatacenter = "dc1"
rt.PrimaryGateways = []string{"a", "b"}
rt.DataDir = dataDir
// server things
rt.ServerMode = true
rt.LeaveOnTerm = false
rt.SkipLeaveOnInt = true
},
},
{
desc: "-protocol",
args: []string{
@ -2878,6 +2901,194 @@ func TestConfigFlagsAndEdgecases(t *testing.T) {
`},
err: "AWS PCA only supports P256 EC curve",
},
{
desc: "connect.enable_mesh_gateway_wan_federation requires connect.enabled",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"connect": {
"enabled": false,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
connect {
enabled = false
enable_mesh_gateway_wan_federation = true
}
`},
err: "'connect.enable_mesh_gateway_wan_federation=true' requires 'connect.enabled=true'",
},
{
desc: "connect.enable_mesh_gateway_wan_federation cannot use -join-wan",
args: []string{
`-data-dir=` + dataDir,
`-join-wan=1.2.3.4`,
},
json: []string{`{
"server": true,
"primary_datacenter": "one",
"datacenter": "one",
"connect": {
"enabled": true,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
server = true
primary_datacenter = "one"
datacenter = "one"
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`},
err: "'start_join_wan' is incompatible with 'connect.enable_mesh_gateway_wan_federation = true'",
},
{
desc: "connect.enable_mesh_gateway_wan_federation cannot use -retry-join-wan",
args: []string{
`-data-dir=` + dataDir,
`-retry-join-wan=1.2.3.4`,
},
json: []string{`{
"server": true,
"primary_datacenter": "one",
"datacenter": "one",
"connect": {
"enabled": true,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
server = true
primary_datacenter = "one"
datacenter = "one"
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`},
err: "'retry_join_wan' is incompatible with 'connect.enable_mesh_gateway_wan_federation = true'",
},
{
desc: "connect.enable_mesh_gateway_wan_federation requires server mode",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"server": false,
"connect": {
"enabled": true,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
server = false
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`},
err: "'connect.enable_mesh_gateway_wan_federation = true' requires 'server = true'",
},
{
desc: "connect.enable_mesh_gateway_wan_federation requires no slashes in node names",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"server": true,
"node_name": "really/why",
"connect": {
"enabled": true,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
server = true
node_name = "really/why"
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`},
err: "'connect.enable_mesh_gateway_wan_federation = true' requires that 'node_name' not contain '/' characters",
},
{
desc: "primary_gateways requires server mode",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"server": false,
"primary_gateways": [ "foo.local", "bar.local" ]
}`},
hcl: []string{`
server = false
primary_gateways = [ "foo.local", "bar.local" ]
`},
err: "'primary_gateways' requires 'server = true'",
},
{
desc: "primary_gateways only works in a secondary datacenter",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"server": true,
"primary_datacenter": "one",
"datacenter": "one",
"primary_gateways": [ "foo.local", "bar.local" ]
}`},
hcl: []string{`
server = true
primary_datacenter = "one"
datacenter = "one"
primary_gateways = [ "foo.local", "bar.local" ]
`},
err: "'primary_gateways' should only be configured in a secondary datacenter",
},
{
desc: "connect.enable_mesh_gateway_wan_federation in secondary with primary_gateways configured",
args: []string{
`-data-dir=` + dataDir,
},
json: []string{`{
"server": true,
"primary_datacenter": "one",
"datacenter": "two",
"primary_gateways": [ "foo.local", "bar.local" ],
"connect": {
"enabled": true,
"enable_mesh_gateway_wan_federation": true
}
}`},
hcl: []string{`
server = true
primary_datacenter = "one"
datacenter = "two"
primary_gateways = [ "foo.local", "bar.local" ]
connect {
enabled = true
enable_mesh_gateway_wan_federation = true
}
`},
patch: func(rt *RuntimeConfig) {
rt.DataDir = dataDir
rt.Datacenter = "two"
rt.PrimaryDatacenter = "one"
rt.ACLDatacenter = "one"
rt.PrimaryGateways = []string{"foo.local", "bar.local"}
rt.ConnectEnabled = true
rt.ConnectMeshGatewayWANFederationEnabled = true
// server things
rt.ServerMode = true
rt.LeaveOnTerm = false
rt.SkipLeaveOnInt = true
},
},
// ------------------------------------------------------------
// ConfigEntry Handling
@ -3795,6 +4006,7 @@ func TestFullConfig(t *testing.T) {
"csr_max_per_second": 100,
"csr_max_concurrent": 2
},
"enable_mesh_gateway_wan_federation": false,
"enabled": true
},
"gossip_lan" : {
@ -3902,6 +4114,8 @@ func TestFullConfig(t *testing.T) {
},
"protocol": 30793,
"primary_datacenter": "ejtmd43d",
"primary_gateways": [ "aej8eeZo", "roh2KahS" ],
"primary_gateways_interval": "18866s",
"raft_protocol": 19016,
"raft_snapshot_threshold": 16384,
"raft_snapshot_interval": "30s",
@ -4424,6 +4638,7 @@ func TestFullConfig(t *testing.T) {
csr_max_per_second = 100.0
csr_max_concurrent = 2.0
}
enable_mesh_gateway_wan_federation = false
enabled = true
}
gossip_lan {
@ -4534,6 +4749,8 @@ func TestFullConfig(t *testing.T) {
}
protocol = 30793
primary_datacenter = "ejtmd43d"
primary_gateways = [ "aej8eeZo", "roh2KahS" ]
primary_gateways_interval = "18866s"
raft_protocol = 19016
raft_snapshot_threshold = 16384
raft_snapshot_interval = "30s"
@ -5148,94 +5365,97 @@ func TestFullConfig(t *testing.T) {
"CSRMaxPerSecond": float64(100),
"CSRMaxConcurrent": float64(2),
},
DNSAddrs: []net.Addr{tcpAddr("93.95.95.81:7001"), udpAddr("93.95.95.81:7001")},
DNSARecordLimit: 29907,
DNSAllowStale: true,
DNSDisableCompression: true,
DNSDomain: "7W1xXSqd",
DNSAltDomain: "1789hsd",
DNSEnableTruncate: true,
DNSMaxStale: 29685 * time.Second,
DNSNodeTTL: 7084 * time.Second,
DNSOnlyPassing: true,
DNSPort: 7001,
DNSRecursorTimeout: 4427 * time.Second,
DNSRecursors: []string{"63.38.39.58", "92.49.18.18"},
DNSSOA: RuntimeSOAConfig{Refresh: 3600, Retry: 600, Expire: 86400, Minttl: 0},
DNSServiceTTL: map[string]time.Duration{"*": 32030 * time.Second},
DNSUDPAnswerLimit: 29909,
DNSNodeMetaTXT: true,
DNSUseCache: true,
DNSCacheMaxAge: 5 * time.Minute,
DataDir: dataDir,
Datacenter: "rzo029wg",
DefaultQueryTime: 16743 * time.Second,
DevMode: true,
DisableAnonymousSignature: true,
DisableCoordinates: true,
DisableHostNodeID: true,
DisableHTTPUnprintableCharFilter: true,
DisableKeyringFile: true,
DisableRemoteExec: true,
DisableUpdateCheck: true,
DiscardCheckOutput: true,
DiscoveryMaxStale: 5 * time.Second,
EnableAgentTLSForChecks: true,
EnableCentralServiceConfig: true,
EnableDebug: true,
EnableRemoteScriptChecks: true,
EnableLocalScriptChecks: true,
EnableSyslog: true,
EnableUI: true,
EncryptKey: "A4wELWqH",
EncryptVerifyIncoming: true,
EncryptVerifyOutgoing: true,
GRPCPort: 4881,
GRPCAddrs: []net.Addr{tcpAddr("32.31.61.91:4881")},
HTTPAddrs: []net.Addr{tcpAddr("83.39.91.39:7999")},
HTTPBlockEndpoints: []string{"RBvAFcGD", "fWOWFznh"},
AllowWriteHTTPFrom: []*net.IPNet{cidr("127.0.0.0/8"), cidr("22.33.44.55/32"), cidr("0.0.0.0/0")},
HTTPPort: 7999,
HTTPResponseHeaders: map[string]string{"M6TKa9NP": "xjuxjOzQ", "JRCrHZed": "rl0mTx81"},
HTTPSAddrs: []net.Addr{tcpAddr("95.17.17.19:15127")},
HTTPMaxConnsPerClient: 9283,
HTTPSHandshakeTimeout: 2391 * time.Millisecond,
HTTPSPort: 15127,
KeyFile: "IEkkwgIA",
KVMaxValueSize: 1234567800000000,
LeaveDrainTime: 8265 * time.Second,
LeaveOnTerm: true,
LogLevel: "k1zo9Spt",
LogJSON: true,
MaxQueryTime: 18237 * time.Second,
NodeID: types.NodeID("AsUIlw99"),
NodeMeta: map[string]string{"5mgGQMBk": "mJLtVMSG", "A7ynFMJB": "0Nx6RGab"},
NodeName: "otlLxGaI",
NonVotingServer: true,
PidFile: "43xN80Km",
PrimaryDatacenter: "ejtmd43d",
RPCAdvertiseAddr: tcpAddr("17.99.29.16:3757"),
RPCBindAddr: tcpAddr("16.99.34.17:3757"),
RPCHandshakeTimeout: 1932 * time.Millisecond,
RPCHoldTimeout: 15707 * time.Second,
RPCProtocol: 30793,
RPCRateLimit: 12029.43,
RPCMaxBurst: 44848,
RPCMaxConnsPerClient: 2954,
RaftProtocol: 19016,
RaftSnapshotThreshold: 16384,
RaftSnapshotInterval: 30 * time.Second,
RaftTrailingLogs: 83749,
ReconnectTimeoutLAN: 23739 * time.Second,
ReconnectTimeoutWAN: 26694 * time.Second,
RejoinAfterLeave: true,
RetryJoinIntervalLAN: 8067 * time.Second,
RetryJoinIntervalWAN: 28866 * time.Second,
RetryJoinLAN: []string{"pbsSFY7U", "l0qLtWij"},
RetryJoinMaxAttemptsLAN: 913,
RetryJoinMaxAttemptsWAN: 23160,
RetryJoinWAN: []string{"PFsR02Ye", "rJdQIhER"},
SegmentName: "BC2NhTDi",
ConnectMeshGatewayWANFederationEnabled: false,
DNSAddrs: []net.Addr{tcpAddr("93.95.95.81:7001"), udpAddr("93.95.95.81:7001")},
DNSARecordLimit: 29907,
DNSAllowStale: true,
DNSDisableCompression: true,
DNSDomain: "7W1xXSqd",
DNSAltDomain: "1789hsd",
DNSEnableTruncate: true,
DNSMaxStale: 29685 * time.Second,
DNSNodeTTL: 7084 * time.Second,
DNSOnlyPassing: true,
DNSPort: 7001,
DNSRecursorTimeout: 4427 * time.Second,
DNSRecursors: []string{"63.38.39.58", "92.49.18.18"},
DNSSOA: RuntimeSOAConfig{Refresh: 3600, Retry: 600, Expire: 86400, Minttl: 0},
DNSServiceTTL: map[string]time.Duration{"*": 32030 * time.Second},
DNSUDPAnswerLimit: 29909,
DNSNodeMetaTXT: true,
DNSUseCache: true,
DNSCacheMaxAge: 5 * time.Minute,
DataDir: dataDir,
Datacenter: "rzo029wg",
DefaultQueryTime: 16743 * time.Second,
DevMode: true,
DisableAnonymousSignature: true,
DisableCoordinates: true,
DisableHostNodeID: true,
DisableHTTPUnprintableCharFilter: true,
DisableKeyringFile: true,
DisableRemoteExec: true,
DisableUpdateCheck: true,
DiscardCheckOutput: true,
DiscoveryMaxStale: 5 * time.Second,
EnableAgentTLSForChecks: true,
EnableCentralServiceConfig: true,
EnableDebug: true,
EnableRemoteScriptChecks: true,
EnableLocalScriptChecks: true,
EnableSyslog: true,
EnableUI: true,
EncryptKey: "A4wELWqH",
EncryptVerifyIncoming: true,
EncryptVerifyOutgoing: true,
GRPCPort: 4881,
GRPCAddrs: []net.Addr{tcpAddr("32.31.61.91:4881")},
HTTPAddrs: []net.Addr{tcpAddr("83.39.91.39:7999")},
HTTPBlockEndpoints: []string{"RBvAFcGD", "fWOWFznh"},
AllowWriteHTTPFrom: []*net.IPNet{cidr("127.0.0.0/8"), cidr("22.33.44.55/32"), cidr("0.0.0.0/0")},
HTTPPort: 7999,
HTTPResponseHeaders: map[string]string{"M6TKa9NP": "xjuxjOzQ", "JRCrHZed": "rl0mTx81"},
HTTPSAddrs: []net.Addr{tcpAddr("95.17.17.19:15127")},
HTTPMaxConnsPerClient: 9283,
HTTPSHandshakeTimeout: 2391 * time.Millisecond,
HTTPSPort: 15127,
KeyFile: "IEkkwgIA",
KVMaxValueSize: 1234567800000000,
LeaveDrainTime: 8265 * time.Second,
LeaveOnTerm: true,
LogLevel: "k1zo9Spt",
LogJSON: true,
MaxQueryTime: 18237 * time.Second,
NodeID: types.NodeID("AsUIlw99"),
NodeMeta: map[string]string{"5mgGQMBk": "mJLtVMSG", "A7ynFMJB": "0Nx6RGab"},
NodeName: "otlLxGaI",
NonVotingServer: true,
PidFile: "43xN80Km",
PrimaryDatacenter: "ejtmd43d",
PrimaryGateways: []string{"aej8eeZo", "roh2KahS"},
PrimaryGatewaysInterval: 18866 * time.Second,
RPCAdvertiseAddr: tcpAddr("17.99.29.16:3757"),
RPCBindAddr: tcpAddr("16.99.34.17:3757"),
RPCHandshakeTimeout: 1932 * time.Millisecond,
RPCHoldTimeout: 15707 * time.Second,
RPCProtocol: 30793,
RPCRateLimit: 12029.43,
RPCMaxBurst: 44848,
RPCMaxConnsPerClient: 2954,
RaftProtocol: 19016,
RaftSnapshotThreshold: 16384,
RaftSnapshotInterval: 30 * time.Second,
RaftTrailingLogs: 83749,
ReconnectTimeoutLAN: 23739 * time.Second,
ReconnectTimeoutWAN: 26694 * time.Second,
RejoinAfterLeave: true,
RetryJoinIntervalLAN: 8067 * time.Second,
RetryJoinIntervalWAN: 28866 * time.Second,
RetryJoinLAN: []string{"pbsSFY7U", "l0qLtWij"},
RetryJoinMaxAttemptsLAN: 913,
RetryJoinMaxAttemptsWAN: 23160,
RetryJoinWAN: []string{"PFsR02Ye", "rJdQIhER"},
SegmentName: "BC2NhTDi",
Segments: []structs.NetworkSegment{
{
Name: "PExYMe2E",
@ -5889,6 +6109,9 @@ func TestSanitize(t *testing.T) {
RetryJoinWAN: []string{
"wan_foo=bar wan_key=baz wan_secret=boom wan_bang=bar",
},
PrimaryGateways: []string{
"pmgw_foo=bar pmgw_key=baz pmgw_secret=boom pmgw_bang=bar",
},
Services: []*structs.ServiceDefinition{
&structs.ServiceDefinition{
Name: "foo",
@ -5991,6 +6214,7 @@ func TestSanitize(t *testing.T) {
"ConnectCAConfig": {},
"ConnectCAProvider": "",
"ConnectEnabled": false,
"ConnectMeshGatewayWANFederationEnabled": false,
"ConnectSidecarMaxPort": 0,
"ConnectSidecarMinPort": 0,
"ConnectTestCALeafRootChangeSpread": "0s",
@ -6097,6 +6321,10 @@ func TestSanitize(t *testing.T) {
"NonVotingServer": false,
"PidFile": "",
"PrimaryDatacenter": "",
"PrimaryGateways": [
"pmgw_foo=bar pmgw_key=baz pmgw_secret=boom pmgw_bang=bar"
],
"PrimaryGatewaysInterval": "0s",
"RPCAdvertiseAddr": "",
"RPCBindAddr": "",
"RPCHandshakeTimeout": "0s",

View File

@ -1320,6 +1320,20 @@ func (f *aclFilter) filterCheckServiceNodes(nodes *structs.CheckServiceNodes) {
*nodes = csn
}
// filterDatacenterCheckServiceNodes is used to filter nodes based on ACL rules.
func (f *aclFilter) filterDatacenterCheckServiceNodes(datacenterNodes *map[string]structs.CheckServiceNodes) {
dn := *datacenterNodes
out := make(map[string]structs.CheckServiceNodes)
for dc, _ := range dn {
nodes := dn[dc]
f.filterCheckServiceNodes(&nodes)
if len(nodes) > 0 {
out[dc] = nodes
}
}
*datacenterNodes = out
}
// filterSessions is used to filter a set of sessions based on ACLs.
func (f *aclFilter) filterSessions(sessions *structs.Sessions) {
s := *sessions
@ -1698,6 +1712,9 @@ func (r *ACLResolver) filterACLWithAuthorizer(authorizer acl.Authorizer, subj in
case *structs.IndexedCheckServiceNodes:
filt.filterCheckServiceNodes(&v.Nodes)
case *structs.DatacenterIndexedCheckServiceNodes:
filt.filterDatacenterCheckServiceNodes(&v.DatacenterNodes)
case *structs.IndexedCoordinates:
filt.filterCoordinates(&v.Coordinates)

View File

@ -11,8 +11,10 @@ import (
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/mitchellh/copystructure"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
@ -2873,6 +2875,106 @@ func TestACL_filterNodes(t *testing.T) {
}
}
func TestACL_filterDatacenterCheckServiceNodes(t *testing.T) {
t.Parallel()
// Create some data.
fixture := map[string]structs.CheckServiceNodes{
"dc1": []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1a", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2a", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
"dc2": []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc2", "gateway1b", "5.6.7.8", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc2", "gateway2b", "8.7.6.5", 1111, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
}
fill := func(t *testing.T) map[string]structs.CheckServiceNodes {
t.Helper()
dup, err := copystructure.Copy(fixture)
require.NoError(t, err)
return dup.(map[string]structs.CheckServiceNodes)
}
// Try permissive filtering.
{
dcNodes := fill(t)
filt := newACLFilter(acl.AllowAll(), nil, true)
filt.filterDatacenterCheckServiceNodes(&dcNodes)
require.Len(t, dcNodes, 2)
require.Equal(t, fill(t), dcNodes)
}
// Try restrictive filtering.
{
dcNodes := fill(t)
filt := newACLFilter(acl.DenyAll(), nil, true)
filt.filterDatacenterCheckServiceNodes(&dcNodes)
require.Len(t, dcNodes, 0)
}
var (
policy *acl.Policy
err error
perms acl.Authorizer
)
// Allowed to see the service but not the node.
policy, err = acl.NewPolicyFromSource("", 0, `
service_prefix "" { policy = "read" }
`, acl.SyntaxCurrent, nil, nil)
require.NoError(t, err)
perms, err = acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
require.NoError(t, err)
{
dcNodes := fill(t)
filt := newACLFilter(perms, nil, true)
filt.filterDatacenterCheckServiceNodes(&dcNodes)
require.Len(t, dcNodes, 0)
}
// Allowed to see the node but not the service.
policy, err = acl.NewPolicyFromSource("", 0, `
node_prefix "" { policy = "read" }
`, acl.SyntaxCurrent, nil, nil)
require.NoError(t, err)
perms, err = acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
require.NoError(t, err)
{
dcNodes := fill(t)
filt := newACLFilter(perms, nil, true)
filt.filterDatacenterCheckServiceNodes(&dcNodes)
require.Len(t, dcNodes, 0)
}
// Allowed to see the service AND the node
policy, err = acl.NewPolicyFromSource("", 0, `
service_prefix "" { policy = "read" }
node_prefix "" { policy = "read" }
`, acl.SyntaxCurrent, nil, nil)
require.NoError(t, err)
perms, err = acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
require.NoError(t, err)
// Now it should go through.
{
dcNodes := fill(t)
filt := newACLFilter(acl.AllowAll(), nil, true)
filt.filterDatacenterCheckServiceNodes(&dcNodes)
require.Len(t, dcNodes, 2)
require.Equal(t, fill(t), dcNodes)
}
}
func TestACL_redactPreparedQueryTokens(t *testing.T) {
t.Parallel()
query := &structs.PreparedQuery{

View File

@ -109,7 +109,7 @@ func (c *Client) RequestAutoEncryptCerts(servers []string, port int, token strin
for _, ip := range ips {
addr := net.TCPAddr{IP: ip, Port: port}
if err = c.connPool.RPC(c.config.Datacenter, &addr, 0, "AutoEncrypt.Sign", true, &args, &reply); err == nil {
if err = c.connPool.RPC(c.config.Datacenter, c.config.NodeName, &addr, 0, "AutoEncrypt.Sign", true, &args, &reply); err == nil {
return &reply, pkPEM, nil
} else {
c.logger.Warn("AutoEncrypt failed", "error", err)

View File

@ -131,12 +131,14 @@ func NewClientLogger(config *Config, logger hclog.InterceptLogger, tlsConfigurat
}
connPool := &pool.ConnPool{
Server: false,
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: clientRPCConnMaxIdle,
MaxStreams: clientMaxStreams,
TLSConfigurator: tlsConfigurator,
ForceTLS: config.VerifyOutgoing,
Datacenter: config.Datacenter,
}
// Create client
@ -311,7 +313,7 @@ TRY:
}
// Make the request.
rpcErr := c.connPool.RPC(c.config.Datacenter, server.Addr, server.Version, method, server.UseTLS, args, reply)
rpcErr := c.connPool.RPC(c.config.Datacenter, server.ShortName, server.Addr, server.Version, method, server.UseTLS, args, reply)
if rpcErr == nil {
return nil
}
@ -359,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, server.UseTLS, args, in, &reply)
snap, err := SnapshotRPC(c.connPool, c.config.Datacenter, server.ShortName, server.Addr, server.UseTLS, args, in, &reply)
if err != nil {
return err
}

View File

@ -399,7 +399,7 @@ func TestClient_RPC_ConsulServerPing(t *testing.T) {
for range servers {
time.Sleep(200 * time.Millisecond)
s := c.routers.FindServer()
ok, err := c.connPool.Ping(s.Datacenter, s.Addr, s.Version, s.UseTLS)
ok, err := c.connPool.Ping(s.Datacenter, s.ShortName, s.Addr, s.Version, s.UseTLS)
if !ok {
t.Errorf("Unable to ping server %v: %s", s.String(), err)
}

View File

@ -370,6 +370,20 @@ type Config struct {
// used to limit the amount of Raft bandwidth used for replication.
ConfigReplicationApplyLimit int
// FederationStateReplicationRate is the max number of replication rounds that can
// be run per second. Note that either 1 or 2 RPCs are used during each replication
// round
FederationStateReplicationRate int
// FederationStateReplicationBurst is how many replication rounds can be bursted after a
// period of idleness
FederationStateReplicationBurst int
// FederationStateReplicationApply limit is the max number of replication-related
// apply operations that we allow during a one second period. This is
// used to limit the amount of Raft bandwidth used for replication.
FederationStateReplicationApplyLimit int
// CoordinateUpdatePeriod controls how long a server batches coordinate
// updates before applying them in a Raft transaction. A larger period
// leads to fewer Raft transactions, but also the stored coordinates
@ -436,6 +450,14 @@ type Config struct {
// ConnectEnabled is whether to enable Connect features such as the CA.
ConnectEnabled bool
// ConnectMeshGatewayWANFederationEnabled determines if wan federation of
// datacenters should exclusively traverse mesh gateways.
ConnectMeshGatewayWANFederationEnabled bool
// DisableFederationStateAntiEntropy solely exists for use in unit tests to
// disable a background routine.
DisableFederationStateAntiEntropy bool
// CAConfig is used to apply the initial Connect CA configuration when
// bootstrapping.
CAConfig *structs.CAConfiguration
@ -509,32 +531,35 @@ func DefaultConfig() *Config {
}
conf := &Config{
Build: version.Version,
Datacenter: DefaultDC,
NodeName: hostname,
RPCAddr: DefaultRPCAddr,
RaftConfig: raft.DefaultConfig(),
SerfLANConfig: lib.SerfDefaultConfig(),
SerfWANConfig: lib.SerfDefaultConfig(),
SerfFloodInterval: 60 * time.Second,
ReconcileInterval: 60 * time.Second,
ProtocolVersion: ProtocolVersion2Compatible,
ACLRoleTTL: 30 * time.Second,
ACLPolicyTTL: 30 * time.Second,
ACLTokenTTL: 30 * time.Second,
ACLDefaultPolicy: "allow",
ACLDownPolicy: "extend-cache",
ACLReplicationRate: 1,
ACLReplicationBurst: 5,
ACLReplicationApplyLimit: 100, // ops / sec
ConfigReplicationRate: 1,
ConfigReplicationBurst: 5,
ConfigReplicationApplyLimit: 100, // ops / sec
TombstoneTTL: 15 * time.Minute,
TombstoneTTLGranularity: 30 * time.Second,
SessionTTLMin: 10 * time.Second,
ACLTokenMinExpirationTTL: 1 * time.Minute,
ACLTokenMaxExpirationTTL: 24 * time.Hour,
Build: version.Version,
Datacenter: DefaultDC,
NodeName: hostname,
RPCAddr: DefaultRPCAddr,
RaftConfig: raft.DefaultConfig(),
SerfLANConfig: lib.SerfDefaultConfig(),
SerfWANConfig: lib.SerfDefaultConfig(),
SerfFloodInterval: 60 * time.Second,
ReconcileInterval: 60 * time.Second,
ProtocolVersion: ProtocolVersion2Compatible,
ACLRoleTTL: 30 * time.Second,
ACLPolicyTTL: 30 * time.Second,
ACLTokenTTL: 30 * time.Second,
ACLDefaultPolicy: "allow",
ACLDownPolicy: "extend-cache",
ACLReplicationRate: 1,
ACLReplicationBurst: 5,
ACLReplicationApplyLimit: 100, // ops / sec
ConfigReplicationRate: 1,
ConfigReplicationBurst: 5,
ConfigReplicationApplyLimit: 100, // ops / sec
FederationStateReplicationRate: 1,
FederationStateReplicationBurst: 5,
FederationStateReplicationApplyLimit: 100, // ops / sec
TombstoneTTL: 15 * time.Minute,
TombstoneTTLGranularity: 30 * time.Second,
SessionTTLMin: 10 * time.Second,
ACLTokenMinExpirationTTL: 1 * time.Minute,
ACLTokenMaxExpirationTTL: 24 * time.Hour,
// These are tuned to provide a total throughput of 128 updates
// per second. If you update these, you should update the client-

View File

@ -35,6 +35,10 @@ func (s *Server) handleEnterpriseRPCConn(rtype pool.RPCType, conn net.Conn, isTL
return false
}
func (s *Server) handleEnterpriseNativeTLSConn(alpnProto string, conn net.Conn) bool {
return false
}
func (s *Server) handleEnterpriseLeave() {
return
}

View File

@ -0,0 +1,176 @@
package consul
import (
"fmt"
"time"
metrics "github.com/armon/go-metrics"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs"
memdb "github.com/hashicorp/go-memdb"
)
// FederationState endpoint is used to manipulate federation states from all
// datacenters.
type FederationState struct {
srv *Server
}
func (c *FederationState) Apply(args *structs.FederationStateRequest, reply *bool) error {
// Ensure that all federation state writes go to the primary datacenter. These will then
// be replicated to all the other datacenters.
args.Datacenter = c.srv.config.PrimaryDatacenter
if done, err := c.srv.forward("FederationState.Apply", args, args, reply); done {
return err
}
defer metrics.MeasureSince([]string{"federation_state", "apply"}, time.Now())
// Fetch the ACL token, if any.
rule, err := c.srv.ResolveToken(args.Token)
if err != nil {
return err
}
if rule != nil && rule.OperatorWrite(nil) != acl.Allow {
return acl.ErrPermissionDenied
}
if args.State == nil || args.State.Datacenter == "" {
return fmt.Errorf("invalid request: missing federation state datacenter")
}
switch args.Op {
case structs.FederationStateUpsert:
if args.State.UpdatedAt.IsZero() {
args.State.UpdatedAt = time.Now().UTC()
}
case structs.FederationStateDelete:
// No validation required.
default:
return fmt.Errorf("Invalid federation state operation: %v", args.Op)
}
resp, err := c.srv.raftApply(structs.FederationStateRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
if respBool, ok := resp.(bool); ok {
*reply = respBool
}
return nil
}
func (c *FederationState) Get(args *structs.FederationStateQuery, reply *structs.FederationStateResponse) error {
if done, err := c.srv.forward("FederationState.Get", args, args, reply); done {
return err
}
defer metrics.MeasureSince([]string{"federation_state", "get"}, time.Now())
// Fetch the ACL token, if any.
rule, err := c.srv.ResolveToken(args.Token)
if err != nil {
return err
}
if rule != nil && rule.OperatorRead(nil) != acl.Allow {
return acl.ErrPermissionDenied
}
return c.srv.blockingQuery(
&args.QueryOptions,
&reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, fedState, err := state.FederationStateGet(ws, args.Datacenter)
if err != nil {
return err
}
reply.Index = index
if fedState == nil {
return nil
}
reply.State = fedState
return nil
})
}
// List is the endpoint meant to be used by consul servers performing
// replication.
func (c *FederationState) List(args *structs.DCSpecificRequest, reply *structs.IndexedFederationStates) error {
if done, err := c.srv.forward("FederationState.List", args, args, reply); done {
return err
}
defer metrics.MeasureSince([]string{"federation_state", "list"}, time.Now())
// Fetch the ACL token, if any.
rule, err := c.srv.ResolveToken(args.Token)
if err != nil {
return err
}
if rule != nil && rule.OperatorRead(nil) != acl.Allow {
return acl.ErrPermissionDenied
}
return c.srv.blockingQuery(
&args.QueryOptions,
&reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, fedStates, err := state.FederationStateList(ws)
if err != nil {
return err
}
if len(fedStates) == 0 {
fedStates = []*structs.FederationState{}
}
reply.Index = index
reply.States = fedStates
return nil
})
}
// ListMeshGateways is the endpoint meant to be used by proxies only interested
// in the discovery info for dialing mesh gateways. Analogous to catalog
// endpoints.
func (c *FederationState) ListMeshGateways(args *structs.DCSpecificRequest, reply *structs.DatacenterIndexedCheckServiceNodes) error {
if done, err := c.srv.forward("FederationState.ListMeshGateways", args, args, reply); done {
return err
}
defer metrics.MeasureSince([]string{"federation_state", "list_mesh_gateways"}, time.Now())
return c.srv.blockingQuery(
&args.QueryOptions,
&reply.QueryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
index, fedStates, err := state.FederationStateList(ws)
if err != nil {
return err
}
dump := make(map[string]structs.CheckServiceNodes)
for i, _ := range fedStates {
fedState := fedStates[i]
csn := fedState.MeshGateways
if len(csn) > 0 {
// We shallow clone this slice so that the filterACL doesn't
// end up manipulating the slice in memdb.
dump[fedState.Datacenter] = csn.ShallowClone()
}
}
reply.Index, reply.DatacenterNodes = index, dump
if err := c.srv.filterACL(args.Token, reply); err != nil {
return err
}
return nil
})
}

View File

@ -0,0 +1,823 @@
package consul
import (
"net/rpc"
"os"
"testing"
"time"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/types"
uuid "github.com/hashicorp/go-uuid"
msgpackrpc "github.com/hashicorp/net-rpc-msgpackrpc"
"github.com/stretchr/testify/require"
)
func TestFederationState_Apply_Upsert(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
codec2 := rpcClient(t, s2)
defer codec2.Close()
testrpc.WaitForLeader(t, s2.RPC, "dc2")
joinWAN(t, s2, s1)
// wait for cross-dc queries to work
testrpc.WaitForLeader(t, s2.RPC, "dc1")
// update the primary with data from a secondary by way of request forwarding
fedState := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec2, "", fedState)
// the previous RPC should not return until the primary has been updated but will return
// before the secondary has the data.
state := s1.fsm.State()
_, fedState2, err := state.FederationStateGet(nil, "dc1")
require.NoError(t, err)
require.NotNil(t, fedState2)
zeroFedStateIndexes(t, fedState2)
require.Equal(t, fedState, fedState2)
retry.Run(t, func(r *retry.R) {
// wait for replication to happen
state := s2.fsm.State()
_, fedState2Again, err := state.FederationStateGet(nil, "dc1")
require.NoError(r, err)
require.NotNil(r, fedState2Again)
// this test is not testing that the federation states that are
// replicated are correct as that's done elsewhere.
})
updated := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway3", "9.9.9.9", 7777, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec2, "", updated)
state = s1.fsm.State()
_, fedState2, err = state.FederationStateGet(nil, "dc1")
require.NoError(t, err)
require.NotNil(t, fedState2)
zeroFedStateIndexes(t, fedState2)
require.Equal(t, updated, fedState2)
}
func TestFederationState_Apply_Upsert_ACLDeny(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
codec := rpcClient(t, s1)
defer codec.Close()
// Create the ACL tokens
opReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `operator = "read"`)
require.NoError(t, err)
opWriteToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `operator = "write"`)
require.NoError(t, err)
expected := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
{ // This should fail since we don't have write perms.
args := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateUpsert,
State: expected,
WriteRequest: structs.WriteRequest{Token: opReadToken.SecretID},
}
out := false
err := msgpackrpc.CallWithCodec(codec, "FederationState.Apply", &args, &out)
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
}
}
{ // This should work.
args := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateUpsert,
State: expected,
WriteRequest: structs.WriteRequest{Token: opWriteToken.SecretID},
}
out := false
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.Apply", &args, &out))
}
// the previous RPC should not return until the primary has been updated but will return
// before the secondary has the data.
state := s1.fsm.State()
_, got, err := state.FederationStateGet(nil, "dc1")
require.NoError(t, err)
require.NotNil(t, got)
zeroFedStateIndexes(t, got)
require.Equal(t, expected, got)
}
func TestFederationState_Get(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
codec := rpcClient(t, s1)
defer codec.Close()
expected := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec, "", expected)
args := structs.FederationStateQuery{
Datacenter: "dc1",
TargetDatacenter: "dc1",
}
var out structs.FederationStateResponse
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.Get", &args, &out))
zeroFedStateIndexes(t, out.State)
require.Equal(t, expected, out.State)
}
func TestFederationState_Get_ACLDeny(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForTestAgent(t, s1.RPC, "dc1")
codec := rpcClient(t, s1)
defer codec.Close()
// Create the ACL tokens
nadaToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
service "foo" { policy = "write" }`)
require.NoError(t, err)
opReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
operator = "read"`)
require.NoError(t, err)
// create some dummy stuff to look up
expected := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec, "root", expected)
{ // This should fail
args := structs.FederationStateQuery{
Datacenter: "dc1",
TargetDatacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: nadaToken.SecretID},
}
var out structs.FederationStateResponse
err := msgpackrpc.CallWithCodec(codec, "FederationState.Get", &args, &out)
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
}
}
{ // This should work
args := structs.FederationStateQuery{
Datacenter: "dc1",
TargetDatacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: opReadToken.SecretID},
}
var out structs.FederationStateResponse
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.Get", &args, &out))
zeroFedStateIndexes(t, out.State)
require.Equal(t, expected, out.State)
}
}
func TestFederationState_List(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
codec2 := rpcClient(t, s2)
defer codec2.Close()
testrpc.WaitForLeader(t, s2.RPC, "dc2")
joinWAN(t, s2, s1)
// wait for cross-dc queries to work
testrpc.WaitForLeader(t, s2.RPC, "dc1")
// create some dummy data
expected := structs.IndexedFederationStates{
States: []*structs.FederationState{
{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
{
Datacenter: "dc2",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc2", "gateway1", "5.6.7.8", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc2", "gateway2", "8.7.6.5", 1111, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
},
}
federationStateUpsert(t, codec, "", expected.States[0])
federationStateUpsert(t, codec, "", expected.States[1])
// we'll also test the other list endpoint at the same time since the setup is nearly the same
expectedMeshGateways := structs.DatacenterIndexedCheckServiceNodes{
DatacenterNodes: map[string]structs.CheckServiceNodes{
"dc1": expected.States[0].MeshGateways,
"dc2": expected.States[1].MeshGateways,
},
}
t.Run("List", func(t *testing.T) {
args := structs.DCSpecificRequest{
Datacenter: "dc1",
}
var out structs.IndexedFederationStates
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.List", &args, &out))
for i, _ := range out.States {
zeroFedStateIndexes(t, out.States[i])
}
require.Equal(t, expected.States, out.States)
})
t.Run("ListMeshGateways", func(t *testing.T) {
args := structs.DCSpecificRequest{
Datacenter: "dc1",
}
var out structs.DatacenterIndexedCheckServiceNodes
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.ListMeshGateways", &args, &out))
require.Equal(t, expectedMeshGateways.DatacenterNodes, out.DatacenterNodes)
})
}
func TestFederationState_List_ACLDeny(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
c.ACLEnforceVersion8 = true // apparently this is still not defaulted to true in server code
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
c.ACLEnforceVersion8 = true // ugh
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
codec2 := rpcClient(t, s2)
defer codec2.Close()
testrpc.WaitForLeader(t, s2.RPC, "dc2")
joinWAN(t, s2, s1)
// wait for cross-dc queries to work
testrpc.WaitForLeader(t, s2.RPC, "dc1")
// Create the ACL tokens
nadaToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", ` `)
require.NoError(t, err)
opReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
operator = "read"`)
require.NoError(t, err)
svcReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
service_prefix "" { policy = "read" }`)
require.NoError(t, err)
nodeReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
node_prefix "" { policy = "read" }`)
require.NoError(t, err)
svcAndNodeReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
service_prefix "" { policy = "read" }
node_prefix "" { policy = "read" }`)
require.NoError(t, err)
// create some dummy data
expected := structs.IndexedFederationStates{
States: []*structs.FederationState{
{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
{
Datacenter: "dc2",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc2", "gateway1", "5.6.7.8", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc2", "gateway2", "8.7.6.5", 1111, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
},
}
federationStateUpsert(t, codec, "root", expected.States[0])
federationStateUpsert(t, codec, "root", expected.States[1])
// we'll also test the other list endpoint at the same time since the setup is nearly the same
expectedMeshGateways := structs.DatacenterIndexedCheckServiceNodes{
DatacenterNodes: map[string]structs.CheckServiceNodes{
"dc1": expected.States[0].MeshGateways,
"dc2": expected.States[1].MeshGateways,
},
}
type tcase struct {
token string
listDenied bool
listEmpty bool
gwListEmpty bool
}
cases := map[string]tcase{
"no token": tcase{
token: "",
listDenied: true,
gwListEmpty: true,
},
"no perms": tcase{
token: nadaToken.SecretID,
listDenied: true,
gwListEmpty: true,
},
"service:read": tcase{
token: svcReadToken.SecretID,
listDenied: true,
gwListEmpty: true,
},
"node:read": tcase{
token: nodeReadToken.SecretID,
listDenied: true,
gwListEmpty: true,
},
"service:read and node:read": tcase{
token: svcAndNodeReadToken.SecretID,
listDenied: true,
},
"operator:read": tcase{
token: opReadToken.SecretID,
gwListEmpty: true,
},
"master token": tcase{
token: "root",
},
}
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
t.Run("List", func(t *testing.T) {
args := structs.DCSpecificRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: tc.token},
}
var out structs.IndexedFederationStates
err := msgpackrpc.CallWithCodec(codec, "FederationState.List", &args, &out)
if tc.listDenied {
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
}
} else if tc.listEmpty {
require.NoError(t, err)
require.Len(t, out.States, 0)
} else {
require.NoError(t, err)
for i, _ := range out.States {
zeroFedStateIndexes(t, out.States[i])
}
require.Equal(t, expected.States, out.States)
}
})
t.Run("ListMeshGateways", func(t *testing.T) {
args := structs.DCSpecificRequest{
Datacenter: "dc1",
QueryOptions: structs.QueryOptions{Token: tc.token},
}
var out structs.DatacenterIndexedCheckServiceNodes
err := msgpackrpc.CallWithCodec(codec, "FederationState.ListMeshGateways", &args, &out)
if tc.gwListEmpty {
require.NoError(t, err)
require.Len(t, out.DatacenterNodes, 0)
} else {
require.NoError(t, err)
require.Equal(t, expectedMeshGateways.DatacenterNodes, out.DatacenterNodes)
}
})
})
}
}
func TestFederationState_Apply_Delete(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
codec := rpcClient(t, s1)
defer codec.Close()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
codec2 := rpcClient(t, s2)
defer codec2.Close()
testrpc.WaitForLeader(t, s2.RPC, "dc2")
joinWAN(t, s2, s1)
// wait for cross-dc queries to work
testrpc.WaitForLeader(t, s2.RPC, "dc1")
// Create a dummy federation state in the state store to look up.
fedState := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec, "", fedState)
// Verify it's there
state := s1.fsm.State()
_, existing, err := state.FederationStateGet(nil, "dc1")
require.NoError(t, err)
zeroFedStateIndexes(t, existing)
require.Equal(t, fedState, existing)
retry.Run(t, func(r *retry.R) {
// wait for it to be replicated into the secondary dc
state := s2.fsm.State()
_, fedState2Again, err := state.FederationStateGet(nil, "dc1")
require.NoError(r, err)
require.NotNil(r, fedState2Again)
})
// send the delete request to dc2 - it should get forwarded to dc1.
args := structs.FederationStateRequest{
Op: structs.FederationStateDelete,
State: fedState,
}
out := false
require.NoError(t, msgpackrpc.CallWithCodec(codec2, "FederationState.Apply", &args, &out))
// Verify the entry was deleted.
_, existing, err = s1.fsm.State().FederationStateGet(nil, "dc1")
require.NoError(t, err)
require.Nil(t, existing)
// verify it gets deleted from the secondary too
retry.Run(t, func(r *retry.R) {
_, existing, err := s2.fsm.State().FederationStateGet(nil, "dc1")
require.NoError(r, err)
require.Nil(r, existing)
})
}
func TestFederationState_Apply_Delete_ACLDeny(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.DisableFederationStateAntiEntropy = true
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
codec := rpcClient(t, s1)
defer codec.Close()
// Create the ACL tokens
opReadToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
operator = "read"`)
require.NoError(t, err)
opWriteToken, err := upsertTestTokenWithPolicyRules(codec, "root", "dc1", `
operator = "write"`)
require.NoError(t, err)
// Create a dummy federation state in the state store to look up.
fedState := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
federationStateUpsert(t, codec, "root", fedState)
{ // This should not work
args := structs.FederationStateRequest{
Op: structs.FederationStateDelete,
State: fedState,
WriteRequest: structs.WriteRequest{Token: opReadToken.SecretID},
}
out := false
err := msgpackrpc.CallWithCodec(codec, "FederationState.Apply", &args, &out)
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
}
}
{ // This should work
args := structs.FederationStateRequest{
Op: structs.FederationStateDelete,
State: fedState,
WriteRequest: structs.WriteRequest{Token: opWriteToken.SecretID},
}
out := false
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.Apply", &args, &out))
}
// Verify the entry was deleted.
state := s1.fsm.State()
_, existing, err := state.FederationStateGet(nil, "dc1")
require.NoError(t, err)
require.Nil(t, existing)
}
func newTestGatewayList(
ip1 string, port1 int, meta1 map[string]string,
ip2 string, port2 int, meta2 map[string]string,
) structs.CheckServiceNodes {
return []structs.CheckServiceNode{
{
Node: &structs.Node{
ID: "664bac9f-4de7-4f1b-ad35-0e5365e8f329",
Node: "gateway1",
Datacenter: "dc1",
Address: ip1,
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Port: port1,
Meta: meta1,
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
{
Node: &structs.Node{
ID: "3fb9a696-8209-4eee-a1f7-48600deb9716",
Node: "gateway2",
Datacenter: "dc1",
Address: ip2,
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Port: port2,
Meta: meta2,
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
}
}
func newTestMeshGatewayNode(
datacenter, node string,
ip string,
port int,
meta map[string]string,
healthStatus string,
) structs.CheckServiceNode {
id, err := uuid.GenerateUUID()
if err != nil {
panic(err)
}
return structs.CheckServiceNode{
Node: &structs.Node{
ID: types.NodeID(id),
Node: node,
Datacenter: datacenter,
Address: ip,
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: port,
Meta: meta,
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: healthStatus,
ServiceID: "mesh-gateway",
},
},
}
}
func federationStateUpsert(t *testing.T, codec rpc.ClientCodec, token string, fedState *structs.FederationState) {
dup := *fedState
fedState2 := &dup
args := structs.FederationStateRequest{
Op: structs.FederationStateUpsert,
State: fedState2,
WriteRequest: structs.WriteRequest{Token: token},
}
out := false
require.NoError(t, msgpackrpc.CallWithCodec(codec, "FederationState.Apply", &args, &out))
require.True(t, out)
}
func zeroFedStateIndexes(t *testing.T, fedState *structs.FederationState) {
require.NotNil(t, fedState)
require.True(t, fedState.PrimaryModifyIndex > 0, "this should be set")
fedState.PrimaryModifyIndex = 0 // zero out so the equality works
fedState.RaftIndex = structs.RaftIndex{} // zero these out so the equality works
}

View File

@ -0,0 +1,201 @@
package consul
import (
"context"
"fmt"
"sort"
"time"
"github.com/hashicorp/consul/agent/structs"
)
type FederationStateReplicator struct {
srv *Server
}
var _ IndexReplicatorDelegate = (*FederationStateReplicator)(nil)
// SingularNoun implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) SingularNoun() string { return "federation state" }
// PluralNoun implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) PluralNoun() string { return "federation states" }
// MetricName implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) MetricName() string { return "federation-state" }
// FetchRemote implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) FetchRemote(lastRemoteIndex uint64) (int, interface{}, uint64, error) {
req := structs.DCSpecificRequest{
Datacenter: r.srv.config.PrimaryDatacenter,
QueryOptions: structs.QueryOptions{
AllowStale: true,
MinQueryIndex: lastRemoteIndex,
Token: r.srv.tokens.ReplicationToken(),
},
}
var response structs.IndexedFederationStates
if err := r.srv.RPC("FederationState.List", &req, &response); err != nil {
return 0, nil, 0, err
}
states := []*structs.FederationState(response.States)
return len(response.States), states, response.QueryMeta.Index, nil
}
// FetchLocal implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) FetchLocal() (int, interface{}, error) {
_, local, err := r.srv.fsm.State().FederationStateList(nil)
if err != nil {
return 0, nil, err
}
return len(local), local, nil
}
// DiffRemoteAndLocalState implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) DiffRemoteAndLocalState(localRaw interface{}, remoteRaw interface{}, lastRemoteIndex uint64) (*IndexReplicatorDiff, error) {
local, ok := localRaw.([]*structs.FederationState)
if !ok {
return nil, fmt.Errorf("invalid type for local federation states: %T", localRaw)
}
remote, ok := remoteRaw.([]*structs.FederationState)
if !ok {
return nil, fmt.Errorf("invalid type for remote federation states: %T", remoteRaw)
}
federationStateSort(local)
federationStateSort(remote)
var deletions []*structs.FederationState
var updates []*structs.FederationState
var localIdx int
var remoteIdx int
for localIdx, remoteIdx = 0, 0; localIdx < len(local) && remoteIdx < len(remote); {
if local[localIdx].Datacenter == remote[remoteIdx].Datacenter {
// fedState is in both the local and remote state - need to check raft indices
if remote[remoteIdx].ModifyIndex > lastRemoteIndex {
updates = append(updates, remote[remoteIdx])
}
// increment both indices when equal
localIdx += 1
remoteIdx += 1
} else if local[localIdx].Datacenter < remote[remoteIdx].Datacenter {
// fedState no longer in remoted state - needs deleting
deletions = append(deletions, local[localIdx])
// increment just the local index
localIdx += 1
} else {
// local state doesn't have this fedState - needs updating
updates = append(updates, remote[remoteIdx])
// increment just the remote index
remoteIdx += 1
}
}
for ; localIdx < len(local); localIdx += 1 {
deletions = append(deletions, local[localIdx])
}
for ; remoteIdx < len(remote); remoteIdx += 1 {
updates = append(updates, remote[remoteIdx])
}
return &IndexReplicatorDiff{
NumDeletions: len(deletions),
Deletions: deletions,
NumUpdates: len(updates),
Updates: updates,
}, nil
}
func federationStateSort(states []*structs.FederationState) {
sort.Slice(states, func(i, j int) bool {
return states[i].Datacenter < states[j].Datacenter
})
}
// PerformDeletions implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) PerformDeletions(ctx context.Context, deletionsRaw interface{}) (exit bool, err error) {
deletions, ok := deletionsRaw.([]*structs.FederationState)
if !ok {
return false, fmt.Errorf("invalid type for federation states deletions list: %T", deletionsRaw)
}
ticker := time.NewTicker(time.Second / time.Duration(r.srv.config.FederationStateReplicationApplyLimit))
defer ticker.Stop()
for i, state := range deletions {
req := structs.FederationStateRequest{
Op: structs.FederationStateDelete,
Datacenter: r.srv.config.Datacenter,
State: state,
}
resp, err := r.srv.raftApply(structs.FederationStateRequestType, &req)
if err != nil {
return false, err
}
if respErr, ok := resp.(error); ok && err != nil {
return false, respErr
}
if i < len(deletions)-1 {
select {
case <-ctx.Done():
return true, nil
case <-ticker.C:
// do nothing - ready for the next batch
}
}
}
return false, nil
}
// PerformUpdates implements IndexReplicatorDelegate.
func (r *FederationStateReplicator) PerformUpdates(ctx context.Context, updatesRaw interface{}) (exit bool, err error) {
updates, ok := updatesRaw.([]*structs.FederationState)
if !ok {
return false, fmt.Errorf("invalid type for federation states update list: %T", updatesRaw)
}
ticker := time.NewTicker(time.Second / time.Duration(r.srv.config.FederationStateReplicationApplyLimit))
defer ticker.Stop()
for i, state := range updates {
dup := *state // lightweight copy
state2 := &dup
// Keep track of the raft modify index at the primary
state2.PrimaryModifyIndex = state.ModifyIndex
req := structs.FederationStateRequest{
Op: structs.FederationStateUpsert,
Datacenter: r.srv.config.Datacenter,
State: state2,
}
resp, err := r.srv.raftApply(structs.FederationStateRequestType, &req)
if err != nil {
return false, err
}
if respErr, ok := resp.(error); ok && err != nil {
return false, respErr
}
if i < len(updates)-1 {
select {
case <-ctx.Done():
return true, nil
case <-ticker.C:
// do nothing - ready for the next batch
}
}
}
return false, nil
}

View File

@ -0,0 +1,144 @@
package consul
import (
"fmt"
"os"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/stretchr/testify/require"
)
func TestReplication_FederationStates(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.PrimaryDatacenter = "dc1"
c.DisableFederationStateAntiEntropy = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
client := rpcClient(t, s1)
defer client.Close()
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.FederationStateReplicationRate = 100
c.FederationStateReplicationBurst = 100
c.FederationStateReplicationApplyLimit = 1000000
c.DisableFederationStateAntiEntropy = true
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Try to join.
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
// Create some new federation states (weird because we're having dc1 update it for the other 50)
var fedStates []*structs.FederationState
for i := 0; i < 50; i++ {
dc := fmt.Sprintf("alt-dc%d", i+1)
ip1 := fmt.Sprintf("1.2.3.%d", i+1)
ip2 := fmt.Sprintf("4.3.2.%d", i+1)
arg := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateUpsert,
State: &structs.FederationState{
Datacenter: dc,
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
dc, "gateway1", ip1, 443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
dc, "gateway2", ip2, 443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
}
out := false
require.NoError(t, s1.RPC("FederationState.Apply", &arg, &out))
fedStates = append(fedStates, arg.State)
}
checkSame := func(t *retry.R) error {
_, remote, err := s1.fsm.State().FederationStateList(nil)
require.NoError(t, err)
_, local, err := s2.fsm.State().FederationStateList(nil)
require.NoError(t, err)
require.Len(t, local, len(remote))
for i, _ := range remote {
// zero out the raft data for future comparisons
remote[i].RaftIndex = structs.RaftIndex{}
local[i].RaftIndex = structs.RaftIndex{}
require.Equal(t, remote[i], local[i])
}
return nil
}
// Wait for the replica to converge.
retry.Run(t, func(r *retry.R) {
checkSame(r)
})
// Update those states
for i := 0; i < 50; i++ {
dc := fmt.Sprintf("alt-dc%d", i+1)
ip1 := fmt.Sprintf("1.2.3.%d", i+1)
ip2 := fmt.Sprintf("4.3.2.%d", i+1)
ip3 := fmt.Sprintf("5.8.9.%d", i+1)
arg := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateUpsert,
State: &structs.FederationState{
Datacenter: dc,
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
dc, "gateway1", ip1, 8443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
dc, "gateway2", ip2, 8443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
dc, "gateway3", ip3, 8443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
},
}
out := false
require.NoError(t, s1.RPC("FederationState.Apply", &arg, &out))
}
// Wait for the replica to converge.
retry.Run(t, func(r *retry.R) {
checkSame(r)
})
for _, fedState := range fedStates {
arg := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateDelete,
State: fedState,
}
out := false
require.NoError(t, s1.RPC("FederationState.Apply", &arg, &out))
}
// Wait for the replica to converge.
retry.Run(t, func(r *retry.R) {
checkSame(r)
})
}

View File

@ -36,6 +36,7 @@ func init() {
registerCommand(structs.ACLBindingRuleDeleteRequestType, (*FSM).applyACLBindingRuleDeleteOperation)
registerCommand(structs.ACLAuthMethodSetRequestType, (*FSM).applyACLAuthMethodSetOperation)
registerCommand(structs.ACLAuthMethodDeleteRequestType, (*FSM).applyACLAuthMethodDeleteOperation)
registerCommand(structs.FederationStateRequestType, (*FSM).applyFederationStateOperation)
}
func (c *FSM) applyRegister(buf []byte, index uint64) interface{} {
@ -542,3 +543,26 @@ func (c *FSM) applyACLAuthMethodDeleteOperation(buf []byte, index uint64) interf
return c.state.ACLAuthMethodBatchDelete(index, req.AuthMethodNames, &req.EnterpriseMeta)
}
func (c *FSM) applyFederationStateOperation(buf []byte, index uint64) interface{} {
var req structs.FederationStateRequest
if err := structs.Decode(buf, &req); err != nil {
panic(fmt.Errorf("failed to decode request: %v", err))
}
switch req.Op {
case structs.FederationStateUpsert:
defer metrics.MeasureSinceWithLabels([]string{"fsm", "federation_state", req.State.Datacenter}, time.Now(),
[]metrics.Label{{Name: "op", Value: "upsert"}})
if err := c.state.FederationStateSet(index, req.State); err != nil {
return err
}
return true
case structs.FederationStateDelete:
defer metrics.MeasureSinceWithLabels([]string{"fsm", "federation_state", req.State.Datacenter}, time.Now(),
[]metrics.Label{{Name: "op", Value: "delete"}})
return c.state.FederationStateDelete(index, req.State.Datacenter)
default:
return fmt.Errorf("invalid federation state operation type: %v", req.Op)
}
}

View File

@ -31,6 +31,7 @@ func init() {
registerRestorer(structs.ACLRoleSetRequestType, restoreRole)
registerRestorer(structs.ACLBindingRuleSetRequestType, restoreBindingRule)
registerRestorer(structs.ACLAuthMethodSetRequestType, restoreAuthMethod)
registerRestorer(structs.FederationStateRequestType, restoreFederationState)
}
func persistOSS(s *snapshot, sink raft.SnapshotSink, encoder *codec.Encoder) error {
@ -70,6 +71,9 @@ func persistOSS(s *snapshot, sink raft.SnapshotSink, encoder *codec.Encoder) err
if err := s.persistConfigEntries(sink, encoder); err != nil {
return err
}
if err := s.persistFederationStates(sink, encoder); err != nil {
return err
}
if err := s.persistIndex(sink, encoder); err != nil {
return err
}
@ -435,6 +439,30 @@ func (s *snapshot) persistConfigEntries(sink raft.SnapshotSink,
return nil
}
func (s *snapshot) persistFederationStates(sink raft.SnapshotSink, encoder *codec.Encoder) error {
fedStates, err := s.state.FederationStates()
if err != nil {
return err
}
for _, fedState := range fedStates {
if _, err := sink.Write([]byte{byte(structs.FederationStateRequestType)}); err != nil {
return err
}
// Encode the entry request without an operation since we don't need it for restoring.
// The request is used for its custom decoding/encoding logic around the ConfigEntry
// interface.
req := &structs.FederationStateRequest{
Op: structs.FederationStateUpsert,
State: fedState,
}
if err := encoder.Encode(req); err != nil {
return err
}
}
return nil
}
func (s *snapshot) persistIndex(sink raft.SnapshotSink, encoder *codec.Encoder) error {
// Get all the indexes
iter, err := s.state.Indexes()
@ -672,3 +700,11 @@ func restoreAuthMethod(header *snapshotHeader, restore *state.Restore, decoder *
}
return restore.ACLAuthMethod(&req)
}
func restoreFederationState(header *snapshotHeader, restore *state.Restore, decoder *codec.Decoder) error {
var req structs.FederationStateRequest
if err := decoder.Decode(&req); err != nil {
return err
}
return restore.FederationState(req.State)
}

View File

@ -244,6 +244,7 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
require.NoError(fsm.state.EnsureConfigEntry(18, serviceConfig, structs.DefaultEnterpriseMeta()))
require.NoError(fsm.state.EnsureConfigEntry(19, proxyConfig, structs.DefaultEnterpriseMeta()))
// Raft Chunking
chunkState := &raftchunking.State{
ChunkMap: make(raftchunking.ChunkMap),
}
@ -274,6 +275,110 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
err = fsm.chunker.RestoreState(chunkState)
require.NoError(err)
// Federation states
fedState1 := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
{
Node: &structs.Node{
ID: "664bac9f-4de7-4f1b-ad35-0e5365e8f329",
Node: "gateway1",
Datacenter: "dc1",
Address: "1.2.3.4",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 1111,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
{
Node: &structs.Node{
ID: "3fb9a696-8209-4eee-a1f7-48600deb9716",
Node: "gateway2",
Datacenter: "dc1",
Address: "9.8.7.6",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 2222,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
},
UpdatedAt: time.Now().UTC(),
}
fedState2 := &structs.FederationState{
Datacenter: "dc2",
MeshGateways: []structs.CheckServiceNode{
{
Node: &structs.Node{
ID: "0f92b02e-9f51-4aa2-861b-4ddbc3492724",
Node: "gateway1",
Datacenter: "dc2",
Address: "8.8.8.8",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 3333,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
{
Node: &structs.Node{
ID: "99a76121-1c3f-4023-88ef-805248beb10b",
Node: "gateway2",
Datacenter: "dc2",
Address: "5.5.5.5",
},
Service: &structs.NodeService{
ID: "mesh-gateway",
Service: "mesh-gateway",
Kind: structs.ServiceKindMeshGateway,
Port: 4444,
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
},
Checks: []*structs.HealthCheck{
{
Name: "web connectivity",
Status: api.HealthPassing,
ServiceID: "mesh-gateway",
},
},
},
},
UpdatedAt: time.Now().UTC(),
}
require.NoError(fsm.state.FederationStateSet(21, fedState1))
require.NoError(fsm.state.FederationStateSet(22, fedState2))
// Snapshot
snap, err := fsm.Snapshot()
if err != nil {
@ -492,6 +597,14 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
require.NoError(err)
assert.Equal(newChunkState, chunkState)
// Verify federation states are restored.
_, fedStateLoaded1, err := fsm2.state.FederationStateGet(nil, "dc1")
require.NoError(err)
assert.Equal(fedState1, fedStateLoaded1)
_, fedStateLoaded2, err := fsm2.state.FederationStateGet(nil, "dc2")
require.NoError(err)
assert.Equal(fedState2, fedStateLoaded2)
// Snapshot
snap, err = fsm2.Snapshot()
if err != nil {

View File

@ -0,0 +1,316 @@
package consul
import (
"errors"
"math/rand"
"sort"
"sync"
"time"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/ipaddr"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/logging"
"github.com/hashicorp/go-hclog"
memdb "github.com/hashicorp/go-memdb"
)
// GatewayLocator assists in selecting an appropriate mesh gateway when wan
// federation via mesh gateways is enabled.
//
// This is exclusively used by the consul server itself when it needs to tunnel
// RPC or gossip through a mesh gateway to reach its ultimate destination.
//
// During secondary datacenter bootstrapping there is a phase where it is
// impossible for mesh gateways in the secondary datacenter to register
// themselves into the catalog to be discovered by the servers, so the servers
// maintain references for the mesh gateways in the primary in addition to its
// own local mesh gateways.
//
// After initial datacenter federation the primary mesh gateways are only used
// in extreme fallback situations (basically re-bootstrapping).
//
// For all other operations a consul server will ALWAYS contact a local mesh
// gateway to ultimately forward the request through a remote mesh gateway to
// reach its destination.
type GatewayLocator struct {
logger hclog.Logger
srv serverDelegate
datacenter string // THIS dc
primaryDatacenter string
// these ONLY contain ones that have the wanfed:1 meta
gatewaysLock sync.Mutex
primaryGateways []string // WAN addrs
localGateways []string // LAN addrs
// primaryMeshGatewayDiscoveredAddresses is the current fallback addresses
// for the mesh gateways in the primary datacenter.
primaryMeshGatewayDiscoveredAddresses []string
primaryMeshGatewayDiscoveredAddressesLock sync.Mutex
// This will be closed the FIRST time we get some gateways populated
primaryGatewaysReadyCh chan struct{}
primaryGatewaysReadyOnce sync.Once
}
// PrimaryMeshGatewayAddressesReadyCh returns a channel that will be closed
// when federation state replication ships back at least one primary mesh
// gateway (not via fallback config).
func (g *GatewayLocator) PrimaryMeshGatewayAddressesReadyCh() <-chan struct{} {
return g.primaryGatewaysReadyCh
}
// PickGateway returns the address for a gateway suitable for reaching the
// provided datacenter.
func (g *GatewayLocator) PickGateway(dc string) string {
item := g.pickGateway(dc == g.primaryDatacenter)
g.logger.Trace("picking gateway for transit", "gateway", item, "source_datacenter", g.datacenter, "dest_datacenter", dc)
return item
}
func (g *GatewayLocator) pickGateway(primary bool) string {
addrs := g.listGateways(primary)
return getRandomItem(addrs)
}
func (g *GatewayLocator) listGateways(primary bool) []string {
g.gatewaysLock.Lock()
defer g.gatewaysLock.Unlock()
var addrs []string
if primary {
addrs = g.primaryGateways
} else {
addrs = g.localGateways
}
if primary && len(addrs) == 0 {
addrs = g.PrimaryGatewayFallbackAddresses()
}
return addrs
}
// RefreshPrimaryGatewayFallbackAddresses is used to update the list of current
// fallback addresses for locating mesh gateways in the primary datacenter.
func (g *GatewayLocator) RefreshPrimaryGatewayFallbackAddresses(addrs []string) {
sort.Strings(addrs)
g.primaryMeshGatewayDiscoveredAddressesLock.Lock()
defer g.primaryMeshGatewayDiscoveredAddressesLock.Unlock()
if !lib.StringSliceEqual(addrs, g.primaryMeshGatewayDiscoveredAddresses) {
g.primaryMeshGatewayDiscoveredAddresses = addrs
g.logger.Info("updated fallback list of primary mesh gateways", "mesh_gateways", addrs)
}
}
// PrimaryGatewayFallbackAddresses returns the current set of discovered
// fallback addresses for the mesh gateways in the primary datacenter.
func (g *GatewayLocator) PrimaryGatewayFallbackAddresses() []string {
g.primaryMeshGatewayDiscoveredAddressesLock.Lock()
defer g.primaryMeshGatewayDiscoveredAddressesLock.Unlock()
out := make([]string, len(g.primaryMeshGatewayDiscoveredAddresses))
copy(out, g.primaryMeshGatewayDiscoveredAddresses)
return out
}
func getRandomItem(items []string) string {
switch len(items) {
case 0:
return ""
case 1:
return items[0]
default:
idx := int(rand.Int31n(int32(len(items))))
return items[idx]
}
}
type serverDelegate interface {
blockingQuery(queryOpts structs.QueryOptionsCompat, queryMeta structs.QueryMetaCompat, fn queryFn) error
PrimaryGatewayFallbackAddresses() []string
IsLeader() bool
LeaderLastContact() time.Time
}
func NewGatewayLocator(
logger hclog.Logger,
srv serverDelegate,
datacenter string,
primaryDatacenter string,
) *GatewayLocator {
return &GatewayLocator{
logger: logger.Named(logging.GatewayLocator),
srv: srv,
datacenter: datacenter,
primaryDatacenter: primaryDatacenter,
primaryGatewaysReadyCh: make(chan struct{}),
}
}
var errGatewayLocalStateNotInitialized = errors.New("local state not initialized")
func (g *GatewayLocator) Run(stopCh <-chan struct{}) {
var lastFetchIndex uint64
retryLoopBackoff(stopCh, func() error {
idx, err := g.runOnce(lastFetchIndex)
if err != nil {
return err
}
lastFetchIndex = idx
return nil
}, func(err error) {
if !errors.Is(err, errGatewayLocalStateNotInitialized) {
g.logger.Error("error tracking primary and local mesh gateways", "error", err)
}
})
}
func (g *GatewayLocator) runOnce(lastFetchIndex uint64) (uint64, error) {
if err := g.checkLocalStateIsReady(); err != nil {
return 0, err
}
// NOTE: we can't do RPC here because we won't have a token so we'll just
// mostly assume that our FSM is caught up enough to answer locally. If
// this has drifted it's no different than a cache that drifts or an
// inconsistent read.
queryOpts := &structs.QueryOptions{
MinQueryIndex: lastFetchIndex,
RequireConsistent: false,
}
var (
results []*structs.FederationState
queryMeta structs.QueryMeta
)
err := g.srv.blockingQuery(
queryOpts,
&queryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
// Get the existing stored version of this config that has replicated down.
// We could phone home to get this but that would incur extra WAN traffic
// when we already have enough information locally to figure it out
// (assuming that our replicator is still functioning).
idx, all, err := state.FederationStateList(ws)
if err != nil {
return err
}
queryMeta.Index = idx
results = all
return nil
})
if err != nil {
return 0, err
}
g.updateFromState(results)
return queryMeta.Index, nil
}
// checkLocalStateIsReady is inlined a bit from (*Server).forward(). We need to
// wait until our own state machine is safe to read from.
func (g *GatewayLocator) checkLocalStateIsReady() error {
// Check if we can allow a stale read, ensure our local DB is initialized
if !g.srv.LeaderLastContact().IsZero() {
return nil // the raft leader talked to us
}
if g.srv.IsLeader() {
return nil // we are the leader
}
return errGatewayLocalStateNotInitialized
}
func (g *GatewayLocator) updateFromState(results []*structs.FederationState) {
var (
local structs.CheckServiceNodes
primary structs.CheckServiceNodes
)
for _, config := range results {
retained := retainGateways(config.MeshGateways)
if config.Datacenter == g.datacenter {
local = retained
}
// NOT else-if because conditionals are not mutually exclusive
if config.Datacenter == g.primaryDatacenter {
primary = retained
}
}
primaryAddrs := renderGatewayAddrs(primary, true)
localAddrs := renderGatewayAddrs(local, false)
g.gatewaysLock.Lock()
defer g.gatewaysLock.Unlock()
changed := false
primaryReady := false
if !lib.StringSliceEqual(g.primaryGateways, primaryAddrs) {
g.primaryGateways = primaryAddrs
primaryReady = len(g.primaryGateways) > 0
changed = true
}
if !lib.StringSliceEqual(g.localGateways, localAddrs) {
g.localGateways = localAddrs
changed = true
}
if changed {
g.logger.Info(
"new cached locations of mesh gateways",
"primary", primaryAddrs,
"local", localAddrs,
)
}
if primaryReady {
g.primaryGatewaysReadyOnce.Do(func() {
close(g.primaryGatewaysReadyCh)
})
}
}
func retainGateways(full structs.CheckServiceNodes) structs.CheckServiceNodes {
out := make([]structs.CheckServiceNode, 0, len(full))
for _, csn := range full {
if csn.Service.Meta[structs.MetaWANFederationKey] != "1" {
continue
}
// only keep healthy ones
ok := true
for _, chk := range csn.Checks {
if chk.Status == api.HealthCritical {
ok = false
}
}
if ok {
out = append(out, csn)
}
}
return out
}
func renderGatewayAddrs(gateways structs.CheckServiceNodes, wan bool) []string {
out := make([]string, 0, len(gateways))
for _, csn := range gateways {
addr, port := csn.BestAddress(wan)
completeAddr := ipaddr.FormatAddressPort(addr, port)
out = append(out, completeAddr)
}
sort.Strings(out)
return out
}

View File

@ -0,0 +1,141 @@
package consul
import (
"testing"
"time"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil"
memdb "github.com/hashicorp/go-memdb"
"github.com/stretchr/testify/require"
)
func TestGatewayLocator(t *testing.T) {
state, err := state.NewStateStore(nil)
require.NoError(t, err)
dc1 := &structs.FederationState{
Datacenter: "dc1",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc1", "gateway1", "1.2.3.4", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc1", "gateway2", "4.3.2.1", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
dc2 := &structs.FederationState{
Datacenter: "dc2",
MeshGateways: []structs.CheckServiceNode{
newTestMeshGatewayNode(
"dc2", "gateway1", "5.6.7.8", 5555, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
newTestMeshGatewayNode(
"dc2", "gateway2", "8.7.6.5", 9999, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
),
},
UpdatedAt: time.Now().UTC(),
}
// Insert data for the dcs
require.NoError(t, state.FederationStateSet(1, dc1))
require.NoError(t, state.FederationStateSet(2, dc2))
t.Run("primary", func(t *testing.T) {
logger := testutil.Logger(t)
tsd := &testServerDelegate{State: state, isLeader: true}
g := NewGatewayLocator(
logger,
tsd,
"dc1",
"dc1",
)
idx, err := g.runOnce(0)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.Len(t, tsd.Calls, 1)
require.Equal(t, []string{
"1.2.3.4:5555",
"4.3.2.1:9999",
}, g.listGateways(false))
require.Equal(t, []string{
"1.2.3.4:5555",
"4.3.2.1:9999",
}, g.listGateways(true))
})
t.Run("secondary", func(t *testing.T) {
logger := testutil.Logger(t)
tsd := &testServerDelegate{State: state, isLeader: true}
g := NewGatewayLocator(
logger,
tsd,
"dc2",
"dc1",
)
idx, err := g.runOnce(0)
require.NoError(t, err)
require.Equal(t, uint64(2), idx)
require.Len(t, tsd.Calls, 1)
require.Equal(t, []string{
"5.6.7.8:5555",
"8.7.6.5:9999",
}, g.listGateways(false))
require.Equal(t, []string{
"1.2.3.4:5555",
"4.3.2.1:9999",
}, g.listGateways(true))
})
}
type testServerDelegate struct {
State *state.Store
FallbackAddrs []string
Calls []uint64
isLeader bool
lastContact time.Time
}
// This is just enough to exercise the logic.
func (d *testServerDelegate) blockingQuery(
queryOpts structs.QueryOptionsCompat,
queryMeta structs.QueryMetaCompat,
fn queryFn,
) error {
minQueryIndex := queryOpts.GetMinQueryIndex()
d.Calls = append(d.Calls, minQueryIndex)
var ws memdb.WatchSet
err := fn(ws, d.State)
if err == nil && queryMeta.GetIndex() < 1 {
queryMeta.SetIndex(1)
}
return err
}
func newFakeStateStore() (*state.Store, error) {
return state.NewStateStore(nil)
}
func (d *testServerDelegate) PrimaryGatewayFallbackAddresses() []string {
return d.FallbackAddrs
}
func (d *testServerDelegate) IsLeader() bool {
return d.isLeader
}
func (d *testServerDelegate) LeaderLastContact() time.Time {
return d.lastContact
}

View File

@ -100,8 +100,10 @@ func joinAddrWAN(s *Server) string {
if s == nil {
panic("no server")
}
name := s.config.NodeName
dc := s.config.Datacenter
port := s.config.SerfWANConfig.MemberlistConfig.BindPort
return fmt.Sprintf("127.0.0.1:%d", port)
return fmt.Sprintf("%s.%s/127.0.0.1:%d", name, dc, port)
}
type clientOrServer interface {

View File

@ -340,6 +340,10 @@ func (s *Server) establishLeadership() error {
s.startConfigReplication()
s.startFederationStateReplication()
s.startFederationStateAntiEntropy()
s.startConnectLeader()
s.setConsistentReadReady()
@ -358,6 +362,10 @@ func (s *Server) revokeLeadership() {
s.revokeEnterpriseLeadership()
s.stopFederationStateAntiEntropy()
s.stopFederationStateReplication()
s.stopConfigReplication()
s.stopConnectLeader()
@ -943,6 +951,20 @@ func (s *Server) stopConfigReplication() {
s.leaderRoutineManager.Stop(configReplicationRoutineName)
}
func (s *Server) startFederationStateReplication() {
if s.config.PrimaryDatacenter == "" || s.config.PrimaryDatacenter == s.config.Datacenter {
// replication shouldn't run in the primary DC
return
}
s.leaderRoutineManager.Start(federationStateReplicationRoutineName, s.federationStateReplicator.Run)
}
func (s *Server) stopFederationStateReplication() {
// will be a no-op when not started
s.leaderRoutineManager.Stop(federationStateReplicationRoutineName)
}
// getOrCreateAutopilotConfig is used to get the autopilot config, initializing it if necessary
func (s *Server) getOrCreateAutopilotConfig() *autopilot.Config {
logger := s.loggers.Named(logging.Autopilot)

View File

@ -0,0 +1,224 @@
package consul
import (
"context"
"fmt"
"time"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/structs"
memdb "github.com/hashicorp/go-memdb"
)
const (
// federationStatePruneInterval is how often we check for stale federation
// states to remove should a datacenter be removed from the WAN.
federationStatePruneInterval = time.Hour
)
func (s *Server) startFederationStateAntiEntropy() {
if s.config.DisableFederationStateAntiEntropy {
return
}
s.leaderRoutineManager.Start(federationStateAntiEntropyRoutineName, s.federationStateAntiEntropySync)
// If this is the primary, then also prune any stale datacenters from the
// list of federation states.
if s.config.PrimaryDatacenter == s.config.Datacenter {
s.leaderRoutineManager.Start(federationStatePruningRoutineName, s.federationStatePruning)
}
}
func (s *Server) stopFederationStateAntiEntropy() {
if s.config.DisableFederationStateAntiEntropy {
return
}
s.leaderRoutineManager.Stop(federationStateAntiEntropyRoutineName)
if s.config.PrimaryDatacenter == s.config.Datacenter {
s.leaderRoutineManager.Stop(federationStatePruningRoutineName)
}
}
func (s *Server) federationStateAntiEntropySync(ctx context.Context) error {
var lastFetchIndex uint64
retryLoopBackoff(ctx.Done(), func() error {
idx, err := s.federationStateAntiEntropyMaybeSync(ctx, lastFetchIndex)
if err != nil {
return err
}
lastFetchIndex = idx
return nil
}, func(err error) {
s.logger.Error("error performing anti-entropy sync of federation state", "error", err)
})
return nil
}
func (s *Server) federationStateAntiEntropyMaybeSync(ctx context.Context, lastFetchIndex uint64) (uint64, error) {
queryOpts := &structs.QueryOptions{
MinQueryIndex: lastFetchIndex,
RequireConsistent: true,
// This is just for a local blocking query so no token is needed.
}
idx, prev, curr, err := s.fetchFederationStateAntiEntropyDetails(queryOpts)
if err != nil {
return 0, err
}
// We should check to see if our context was cancelled while we were blocked.
select {
case <-ctx.Done():
return 0, ctx.Err()
default:
}
if prev != nil && prev.IsSame(curr) {
s.logger.Trace("federation state anti-entropy sync skipped; already up to date")
return idx, nil
}
if err := s.updateOurFederationState(curr); err != nil {
return 0, fmt.Errorf("error performing federation state anti-entropy sync: %v", err)
}
s.logger.Info("federation state anti-entropy synced")
return idx, nil
}
func (s *Server) updateOurFederationState(curr *structs.FederationState) error {
if curr.Datacenter != s.config.Datacenter { // sanity check
return fmt.Errorf("cannot use this mechanism to update federation states for other datacenters")
}
curr.UpdatedAt = time.Now().UTC()
args := structs.FederationStateRequest{
Op: structs.FederationStateUpsert,
State: curr,
}
if s.config.Datacenter == s.config.PrimaryDatacenter {
// We are the primary, so we can't do an RPC as we don't have a replication token.
resp, err := s.raftApply(structs.FederationStateRequestType, args)
if err != nil {
return err
}
if respErr, ok := resp.(error); ok {
return respErr
}
} else {
args.WriteRequest = structs.WriteRequest{
Token: s.tokens.ReplicationToken(),
}
ignored := false
if err := s.forwardDC("FederationState.Apply", s.config.PrimaryDatacenter, &args, &ignored); err != nil {
return err
}
}
return nil
}
func (s *Server) fetchFederationStateAntiEntropyDetails(
queryOpts *structs.QueryOptions,
) (uint64, *structs.FederationState, *structs.FederationState, error) {
var (
prevFedState, currFedState *structs.FederationState
queryMeta structs.QueryMeta
)
err := s.blockingQuery(
queryOpts,
&queryMeta,
func(ws memdb.WatchSet, state *state.Store) error {
// Get the existing stored version of this FedState that has replicated down.
// We could phone home to get this but that would incur extra WAN traffic
// when we already have enough information locally to figure it out
// (assuming that our replicator is still functioning).
idx1, prev, err := state.FederationStateGet(ws, s.config.Datacenter)
if err != nil {
return err
}
// Fetch our current list of all mesh gateways.
entMeta := structs.WildcardEnterpriseMeta()
idx2, raw, err := state.ServiceDump(ws, structs.ServiceKindMeshGateway, true, entMeta)
if err != nil {
return err
}
curr := &structs.FederationState{
Datacenter: s.config.Datacenter,
MeshGateways: raw,
}
// Compute the maximum index seen.
if idx2 > idx1 {
queryMeta.Index = idx2
} else {
queryMeta.Index = idx1
}
prevFedState = prev
currFedState = curr
return nil
})
if err != nil {
return 0, nil, nil, err
}
return queryMeta.Index, prevFedState, currFedState, nil
}
func (s *Server) federationStatePruning(ctx context.Context) error {
ticker := time.NewTicker(federationStatePruneInterval)
defer ticker.Stop()
for {
select {
case <-ctx.Done():
return nil
case <-ticker.C:
if err := s.pruneStaleFederationStates(); err != nil {
s.logger.Error("error pruning stale federation states", "error", err)
}
}
}
}
func (s *Server) pruneStaleFederationStates() error {
state := s.fsm.State()
_, fedStates, err := state.FederationStateList(nil)
if err != nil {
return err
}
for _, fedState := range fedStates {
dc := fedState.Datacenter
if s.router.HasDatacenter(dc) {
continue
}
s.logger.Info("pruning stale federation state", "datacenter", dc)
req := structs.FederationStateRequest{
Op: structs.FederationStateDelete,
State: &structs.FederationState{
Datacenter: dc,
},
}
resp, err := s.raftApply(structs.FederationStateRequestType, &req)
if err != nil {
return fmt.Errorf("Failed to delete federation state %s: %v", dc, err)
}
if respErr, ok := resp.(error); ok && err != nil {
return fmt.Errorf("Failed to delete federation state %s: %v", dc, respErr)
}
}
return nil
}

View File

@ -0,0 +1,351 @@
package consul
import (
"os"
"testing"
"time"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/sdk/testutil/retry"
"github.com/hashicorp/consul/testrpc"
"github.com/stretchr/testify/require"
)
func TestLeader_FederationStateAntiEntropy_BlockingQuery(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.FederationStateReplicationRate = 100
c.FederationStateReplicationBurst = 100
c.FederationStateReplicationApplyLimit = 1000000
c.DisableFederationStateAntiEntropy = true
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Try to join.
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
checkSame := func(t *testing.T, expectN, expectGatewaysInDC2 int) {
t.Helper()
retry.Run(t, func(r *retry.R) {
_, remote, err := s1.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, remote, expectN)
_, local, err := s2.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, local, expectN)
var fs2 *structs.FederationState
for _, fs := range local {
if fs.Datacenter == "dc2" {
fs2 = fs
break
}
}
if expectGatewaysInDC2 < 0 {
require.Nil(r, fs2)
} else {
require.NotNil(r, fs2)
require.Len(r, fs2.MeshGateways, expectGatewaysInDC2)
}
})
}
gatewayCSN1 := newTestMeshGatewayNode(
"dc2", "gateway1", "1.2.3.4", 443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
)
gatewayCSN2 := newTestMeshGatewayNode(
"dc2", "gateway2", "4.3.2.1", 443, map[string]string{structs.MetaWANFederationKey: "1"}, api.HealthPassing,
)
// populate with some stuff
makeFedState := func(t *testing.T, dc string, csn ...structs.CheckServiceNode) {
t.Helper()
arg := structs.FederationStateRequest{
Datacenter: "dc1",
Op: structs.FederationStateUpsert,
State: &structs.FederationState{
Datacenter: dc,
MeshGateways: csn,
UpdatedAt: time.Now().UTC(),
},
}
out := false
require.NoError(t, s1.RPC("FederationState.Apply", &arg, &out))
}
makeGateways := func(t *testing.T, csn structs.CheckServiceNode) {
t.Helper()
const dc = "dc2"
arg := structs.RegisterRequest{
Datacenter: csn.Node.Datacenter,
Node: csn.Node.Node,
Address: csn.Node.Address,
Service: csn.Service,
Checks: csn.Checks,
}
var out struct{}
require.NoError(t, s2.RPC("Catalog.Register", &arg, &out))
}
type result struct {
idx uint64
prev, curr *structs.FederationState
err error
}
blockAgain := func(last uint64) <-chan result {
ch := make(chan result, 1)
go func() {
var res result
res.idx, res.prev, res.curr, res.err = s2.fetchFederationStateAntiEntropyDetails(&structs.QueryOptions{
MinQueryIndex: last,
RequireConsistent: true,
})
ch <- res
}()
return ch
}
// wait for the primary to do one round of AE and replicate it
checkSame(t, 1, -1)
// // wait for change to be reflected as well
// makeFedState(t, "dc2")
// checkSame(t, 1)
// Do the initial fetch (len0 local gateways, upstream has nil fedstate)
res0 := <-blockAgain(0)
require.NoError(t, res0.err)
ch := blockAgain(res0.idx)
// bump the local mesh gateways; should unblock query
makeGateways(t, gatewayCSN1)
res1 := <-ch
require.NoError(t, res1.err)
require.NotEqual(t, res1.idx, res0.idx)
require.Nil(t, res1.prev)
require.Len(t, res1.curr.MeshGateways, 1)
checkSame(t, 1, -1) // no fed state update yet
ch = blockAgain(res1.idx)
// do manual AE
makeFedState(t, "dc2", gatewayCSN1)
res2 := <-ch
require.NoError(t, res2.err)
require.NotEqual(t, res2.idx, res1.idx)
require.Len(t, res2.prev.MeshGateways, 1)
require.Len(t, res2.curr.MeshGateways, 1)
checkSame(t, 2, 1)
ch = blockAgain(res2.idx)
// add another local mesh gateway
makeGateways(t, gatewayCSN2)
res3 := <-ch
require.NoError(t, res3.err)
require.NotEqual(t, res3.idx, res2.idx)
require.Len(t, res3.prev.MeshGateways, 1)
require.Len(t, res3.curr.MeshGateways, 2)
checkSame(t, 2, 1)
ch = blockAgain(res3.idx)
// do manual AE
makeFedState(t, "dc2", gatewayCSN1, gatewayCSN2)
res4 := <-ch
require.NoError(t, res4.err)
require.NotEqual(t, res4.idx, res3.idx)
require.Len(t, res4.prev.MeshGateways, 2)
require.Len(t, res4.curr.MeshGateways, 2)
checkSame(t, 2, 2)
}
func TestLeader_FederationStateAntiEntropyPruning(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.PrimaryDatacenter = "dc1"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
client := rpcClient(t, s1)
defer client.Close()
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Try to join.
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
checkSame := func(r *retry.R) error {
_, remote, err := s1.fsm.State().FederationStateList(nil)
require.NoError(r, err)
_, local, err := s2.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, remote, 2)
require.Len(r, local, 2)
for i, _ := range remote {
// zero out the raft data for future comparisons
remote[i].RaftIndex = structs.RaftIndex{}
local[i].RaftIndex = structs.RaftIndex{}
require.Equal(r, remote[i], local[i])
}
return nil
}
// Wait for the replica to converge.
retry.Run(t, func(r *retry.R) {
checkSame(r)
})
// Now leave and shutdown dc2.
require.NoError(t, s2.Leave())
require.NoError(t, s2.Shutdown())
// Wait until we know the router is updated.
retry.Run(t, func(r *retry.R) {
dcs := s1.router.GetDatacenters()
require.Len(r, dcs, 1)
require.Equal(r, "dc1", dcs[0])
})
// Since the background routine is going to run every hour, it likely is
// not going to run during this test, so it's safe to directly invoke the
// core method.
require.NoError(t, s1.pruneStaleFederationStates())
// Wait for dc2 to drop out.
retry.Run(t, func(r *retry.R) {
_, mine, err := s1.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, mine, 1)
require.Equal(r, "dc1", mine[0].Datacenter)
})
}
func TestLeader_FederationStateAntiEntropyPruning_ACLDeny(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.PrimaryDatacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
testrpc.WaitForLeader(t, s1.RPC, "dc1")
client := rpcClient(t, s1)
defer client.Close()
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.ACLDatacenter = "dc1"
c.ACLsEnabled = true
c.ACLMasterToken = "root"
c.ACLDefaultPolicy = "deny"
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
defer os.RemoveAll(dir2)
defer s2.Shutdown()
// Try to join.
joinWAN(t, s2, s1)
testrpc.WaitForLeader(t, s1.RPC, "dc1")
testrpc.WaitForLeader(t, s1.RPC, "dc2")
// Create the ACL token.
opWriteToken, err := upsertTestTokenWithPolicyRules(client, "root", "dc1", `operator = "write"`)
require.NoError(t, err)
require.True(t, s1.tokens.UpdateReplicationToken(opWriteToken.SecretID, token.TokenSourceAPI))
require.True(t, s2.tokens.UpdateReplicationToken(opWriteToken.SecretID, token.TokenSourceAPI))
checkSame := func(r *retry.R) error {
_, remote, err := s1.fsm.State().FederationStateList(nil)
require.NoError(r, err)
_, local, err := s2.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, remote, 2)
require.Len(r, local, 2)
for i, _ := range remote {
// zero out the raft data for future comparisons
remote[i].RaftIndex = structs.RaftIndex{}
local[i].RaftIndex = structs.RaftIndex{}
require.Equal(r, remote[i], local[i])
}
return nil
}
// Wait for the replica to converge.
retry.Run(t, func(r *retry.R) {
checkSame(r)
})
// Now leave and shutdown dc2.
require.NoError(t, s2.Leave())
require.NoError(t, s2.Shutdown())
// Wait until we know the router is updated.
retry.Run(t, func(r *retry.R) {
dcs := s1.router.GetDatacenters()
require.Len(r, dcs, 1)
require.Equal(r, "dc1", dcs[0])
})
// Since the background routine is going to run every hour, it likely is
// not going to run during this test, so it's safe to directly invoke the
// core method.
require.NoError(t, s1.pruneStaleFederationStates())
// Wait for dc2 to drop out.
retry.Run(t, func(r *retry.R) {
_, mine, err := s1.fsm.State().FederationStateList(nil)
require.NoError(r, err)
require.Len(r, mine, 1)
require.Equal(r, "dc1", mine[0].Datacenter)
})
}

View File

@ -106,6 +106,7 @@ func (m *LeaderRoutineManager) Stop(name string) error {
m.logger.Debug("stopping routine", "routine", name)
instance.cancel()
delete(m.routines, name)
return nil
}
@ -122,6 +123,6 @@ func (m *LeaderRoutineManager) StopAll() {
routine.cancel()
}
// just whipe out the entire map
// just wipe out the entire map
m.routines = make(map[string]*leaderRoutine)
}

View File

@ -184,7 +184,7 @@ func parseService(svc *structs.ServiceQuery) error {
}
// Make sure the metadata filters are valid
if err := structs.ValidateMetadata(svc.NodeMeta, true); err != nil {
if err := structs.ValidateNodeMetadata(svc.NodeMeta, true); err != nil {
return err
}

View File

@ -2,6 +2,7 @@ package consul
import (
"crypto/tls"
"encoding/binary"
"errors"
"fmt"
"io"
@ -12,6 +13,7 @@ import (
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/consul/wanfed"
"github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/pool"
"github.com/hashicorp/consul/agent/structs"
@ -89,10 +91,6 @@ func logConn(conn net.Conn) string {
// handleConn is used to determine if this is a Raft or
// Consul type RPC connection and invoke the correct handler
func (s *Server) handleConn(conn net.Conn, isTLS bool) {
// Read a single byte
buf := make([]byte, 1)
// Limit how long the client can hold the connection open before they send the
// magic byte (and authenticate when mTLS is enabled). If `isTLS == true` then
// this also enforces a timeout on how long it takes for the handshake to
@ -100,6 +98,34 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) {
if s.config.RPCHandshakeTimeout > 0 {
conn.SetReadDeadline(time.Now().Add(s.config.RPCHandshakeTimeout))
}
if !isTLS && s.tlsConfigurator.MutualTLSCapable() {
// See if actually this is native TLS multiplexed onto the old
// "type-byte" system.
peekedConn, nativeTLS, err := pool.PeekForTLS(conn)
if err != nil {
if err != io.EOF {
s.rpcLogger().Error(
"failed to read first byte",
"conn", logConn(conn),
"error", err,
)
}
conn.Close()
return
}
if nativeTLS {
s.handleNativeTLS(peekedConn)
return
}
conn = peekedConn
}
// Read a single byte
buf := make([]byte, 1)
if _, err := conn.Read(buf); err != nil {
if err != io.EOF {
s.rpcLogger().Error("failed to read byte",
@ -171,6 +197,97 @@ func (s *Server) handleConn(conn net.Conn, isTLS bool) {
}
}
func (s *Server) handleNativeTLS(conn net.Conn) {
s.rpcLogger().Trace(
"detected actual TLS over RPC port",
"conn", logConn(conn),
)
tlscfg := s.tlsConfigurator.IncomingALPNRPCConfig(pool.RPCNextProtos)
tlsConn := tls.Server(conn, tlscfg)
// Force the handshake to conclude.
if err := tlsConn.Handshake(); err != nil {
s.rpcLogger().Error(
"TLS handshake failed",
"conn", logConn(conn),
"error", err,
)
conn.Close()
return
}
// Reset the deadline as we aren't sure what is expected next - it depends on
// the protocol.
if s.config.RPCHandshakeTimeout > 0 {
conn.SetReadDeadline(time.Time{})
}
var (
cs = tlsConn.ConnectionState()
sni = cs.ServerName
nextProto = cs.NegotiatedProtocol
transport = s.memberlistTransportWAN
)
s.rpcLogger().Trace(
"accepted nativeTLS RPC",
"sni", sni,
"protocol", nextProto,
"conn", logConn(conn),
)
switch nextProto {
case pool.ALPN_RPCConsul:
s.handleConsulConn(tlsConn)
case pool.ALPN_RPCRaft:
metrics.IncrCounter([]string{"rpc", "raft_handoff"}, 1)
s.raftLayer.Handoff(tlsConn)
case pool.ALPN_RPCMultiplexV2:
s.handleMultiplexV2(tlsConn)
case pool.ALPN_RPCSnapshot:
s.handleSnapshotConn(tlsConn)
case pool.ALPN_WANGossipPacket:
if err := s.handleALPN_WANGossipPacketStream(tlsConn); err != nil && err != io.EOF {
s.rpcLogger().Error(
"failed to ingest RPC",
"sni", sni,
"protocol", nextProto,
"conn", logConn(conn),
"error", err,
)
}
case pool.ALPN_WANGossipStream:
// No need to defer the conn.Close() here, the Ingest methods do that.
if err := transport.IngestStream(tlsConn); err != nil {
s.rpcLogger().Error(
"failed to ingest RPC",
"sni", sni,
"protocol", nextProto,
"conn", logConn(conn),
"error", err,
)
}
default:
if !s.handleEnterpriseNativeTLSConn(nextProto, conn) {
s.rpcLogger().Error(
"discarding RPC for unknown negotiated protocol",
"failed to ingest RPC",
"protocol", nextProto,
"conn", logConn(conn),
)
conn.Close()
}
}
}
// handleMultiplexV2 is used to multiplex a single incoming connection
// using the Yamux multiplexer
func (s *Server) handleMultiplexV2(conn net.Conn) {
@ -257,6 +374,70 @@ func (s *Server) handleSnapshotConn(conn net.Conn) {
}()
}
func (s *Server) handleALPN_WANGossipPacketStream(conn net.Conn) error {
defer conn.Close()
transport := s.memberlistTransportWAN
for {
select {
case <-s.shutdownCh:
return nil
default:
}
// Note: if we need to change this format to have additional header
// information we can just negotiate a different ALPN protocol instead
// of needing any sort of version field here.
prefixLen, err := readUint32(conn, wanfed.GossipPacketMaxIdleTime)
if err != nil {
return err
}
// Avoid a memory exhaustion DOS vector here by capping how large this
// packet can be to something reasonable.
if prefixLen > wanfed.GossipPacketMaxByteSize {
return fmt.Errorf("gossip packet size %d exceeds threshold of %d", prefixLen, wanfed.GossipPacketMaxByteSize)
}
lc := &limitedConn{
Conn: conn,
lr: io.LimitReader(conn, int64(prefixLen)),
}
if err := transport.IngestPacket(lc, conn.RemoteAddr(), time.Now(), false); err != nil {
return err
}
}
}
func readUint32(conn net.Conn, timeout time.Duration) (uint32, error) {
// Since requests are framed we can easily just set a deadline on
// reading that frame and then disable it for the rest of the body.
if err := conn.SetReadDeadline(time.Now().Add(timeout)); err != nil {
return 0, err
}
var v uint32
if err := binary.Read(conn, binary.BigEndian, &v); err != nil {
return 0, err
}
if err := conn.SetReadDeadline(time.Time{}); err != nil {
return 0, err
}
return v, nil
}
type limitedConn struct {
net.Conn
lr io.Reader
}
func (c *limitedConn) Read(b []byte) (n int, err error) {
return c.lr.Read(b)
}
// canRetry returns true if the given situation is safe for a retry.
func canRetry(args interface{}, err error) bool {
// No leader errors are always safe to retry since no state could have
@ -317,7 +498,7 @@ CHECK_LEADER:
// Handle the case of a known leader
rpcErr := structs.ErrNoLeader
if leader != nil {
rpcErr = s.connPool.RPC(s.config.Datacenter, leader.Addr,
rpcErr = s.connPool.RPC(s.config.Datacenter, leader.ShortName, leader.Addr,
leader.Version, method, leader.UseTLS, args, reply)
if rpcErr != nil && canRetry(info, rpcErr) {
goto RETRY
@ -383,7 +564,7 @@ func (s *Server) forwardDC(method, dc string, args interface{}, reply interface{
metrics.IncrCounterWithLabels([]string{"rpc", "cross-dc"}, 1,
[]metrics.Label{{Name: "datacenter", Value: dc}})
if err := s.connPool.RPC(dc, server.Addr, server.Version, method, server.UseTLS, args, reply); err != nil {
if err := s.connPool.RPC(dc, server.ShortName, server.Addr, server.Version, method, server.UseTLS, args, reply); err != nil {
manager.NotifyFailedServer(server)
s.rpcLogger().Error("RPC failed to server in DC",
"server", server.Addr,

View File

@ -2,9 +2,12 @@ package consul
import (
"bytes"
"encoding/binary"
"math"
"net"
"os"
"strings"
"sync"
"testing"
"time"
@ -652,3 +655,80 @@ func TestRPC_RPCMaxConnsPerClient(t *testing.T) {
})
}
}
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)
})
}
}

View File

@ -35,6 +35,7 @@ import (
connlimit "github.com/hashicorp/go-connlimit"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/memberlist"
"github.com/hashicorp/raft"
raftboltdb "github.com/hashicorp/raft-boltdb"
"github.com/hashicorp/serf/serf"
@ -90,17 +91,20 @@ const (
)
const (
legacyACLReplicationRoutineName = "legacy ACL replication"
aclPolicyReplicationRoutineName = "ACL policy replication"
aclRoleReplicationRoutineName = "ACL role replication"
aclTokenReplicationRoutineName = "ACL token replication"
aclTokenReapingRoutineName = "acl token reaping"
aclUpgradeRoutineName = "legacy ACL token upgrade"
caRootPruningRoutineName = "CA root pruning"
configReplicationRoutineName = "config entry replication"
intentionReplicationRoutineName = "intention replication"
secondaryCARootWatchRoutineName = "secondary CA roots watch"
secondaryCertRenewWatchRoutineName = "secondary cert renew watch"
legacyACLReplicationRoutineName = "legacy ACL replication"
aclPolicyReplicationRoutineName = "ACL policy replication"
aclRoleReplicationRoutineName = "ACL role replication"
aclTokenReplicationRoutineName = "ACL token replication"
aclTokenReapingRoutineName = "acl token reaping"
aclUpgradeRoutineName = "legacy ACL token upgrade"
caRootPruningRoutineName = "CA root pruning"
configReplicationRoutineName = "config entry replication"
federationStateReplicationRoutineName = "federation state replication"
federationStateAntiEntropyRoutineName = "federation state anti-entropy"
federationStatePruningRoutineName = "federation state pruning"
intentionReplicationRoutineName = "intention replication"
secondaryCARootWatchRoutineName = "secondary CA roots watch"
secondaryCertRenewWatchRoutineName = "secondary cert renew watch"
)
var (
@ -153,6 +157,10 @@ type Server struct {
// centralized config
configReplicator *Replicator
// federationStateReplicator is used to manage the leaders replication routines for
// federation states
federationStateReplicator *Replicator
// tokens holds ACL tokens initially from the configuration, but can
// be updated at runtime, so should always be used instead of going to
// the configuration directly.
@ -240,7 +248,9 @@ type Server struct {
// serfWAN is the Serf cluster maintained between DC's
// which SHOULD only consist of Consul servers
serfWAN *serf.Serf
serfWAN *serf.Serf
memberlistTransportWAN memberlist.IngestionAwareTransport
gatewayLocator *GatewayLocator
// serverLookup tracks server consuls in the local datacenter.
// Used to do leader forwarding and provide fast lookup by server id and address
@ -358,12 +368,14 @@ func NewServerLogger(config *Config, logger hclog.InterceptLogger, tokens *token
shutdownCh := make(chan struct{})
connPool := &pool.ConnPool{
Server: true,
SrcAddr: config.RPCSrcAddr,
LogOutput: config.LogOutput,
MaxTime: serverRPCCache,
MaxStreams: serverMaxStreams,
TLSConfigurator: tlsConfigurator,
ForceTLS: config.VerifyOutgoing,
Datacenter: config.Datacenter,
}
serverLogger := logger.NamedIntercept(logging.ConsulServer)
@ -393,6 +405,16 @@ func NewServerLogger(config *Config, logger hclog.InterceptLogger, tokens *token
aclAuthMethodValidators: authmethod.NewCache(),
}
if s.config.ConnectMeshGatewayWANFederationEnabled {
s.gatewayLocator = NewGatewayLocator(
s.logger,
s,
s.config.Datacenter,
s.config.PrimaryDatacenter,
)
s.connPool.GatewayResolver = s.gatewayLocator.PickGateway
}
// Initialize enterprise specific server functionality
if err := s.initEnterprise(); err != nil {
s.Shutdown()
@ -414,6 +436,22 @@ func NewServerLogger(config *Config, logger hclog.InterceptLogger, tokens *token
return nil, err
}
federationStateReplicatorConfig := ReplicatorConfig{
Name: logging.FederationState,
Delegate: &IndexReplicator{
Delegate: &FederationStateReplicator{srv: s},
Logger: s.logger,
},
Rate: s.config.FederationStateReplicationRate,
Burst: s.config.FederationStateReplicationBurst,
Logger: logger,
}
s.federationStateReplicator, err = NewReplicator(&federationStateReplicatorConfig)
if err != nil {
s.Shutdown()
return nil, err
}
// Initialize the stats fetcher that autopilot will use.
s.statsFetcher = NewStatsFetcher(logger, s.connPool, s.config.Datacenter)
@ -456,6 +494,10 @@ func NewServerLogger(config *Config, logger hclog.InterceptLogger, tokens *token
go s.trackAutoEncryptCARoots()
}
if s.gatewayLocator != nil {
go s.gatewayLocator.Run(s.shutdownCh)
}
// Serf and dynamic bind ports
//
// The LAN serf cluster announces the port of the WAN serf cluster
@ -474,6 +516,11 @@ func NewServerLogger(config *Config, logger hclog.InterceptLogger, tokens *token
s.Shutdown()
return nil, fmt.Errorf("Failed to start WAN Serf: %v", err)
}
// This is always a *memberlist.NetTransport or something which wraps
// it which satisfies this interface.
s.memberlistTransportWAN = config.SerfWANConfig.MemberlistConfig.Transport.(memberlist.IngestionAwareTransport)
// See big comment above why we are doing this.
if serfBindPortWAN == 0 {
serfBindPortWAN = config.SerfWANConfig.MemberlistConfig.BindPort
@ -777,6 +824,7 @@ func (s *Server) setupRPC() error {
return err
}
s.Listener = ln
if s.config.NotifyListen != nil {
s.config.NotifyListen()
}
@ -1012,6 +1060,41 @@ func (s *Server) JoinWAN(addrs []string) (int, error) {
return s.serfWAN.Join(addrs, true)
}
// PrimaryMeshGatewayAddressesReadyCh returns a channel that will be closed
// when federation state replication ships back at least one primary mesh
// gateway (not via fallback config).
func (s *Server) PrimaryMeshGatewayAddressesReadyCh() <-chan struct{} {
if s.gatewayLocator == nil {
return nil
}
return s.gatewayLocator.PrimaryMeshGatewayAddressesReadyCh()
}
// PickRandomMeshGatewaySuitableForDialing is a convenience function used for writing tests.
func (s *Server) PickRandomMeshGatewaySuitableForDialing(dc string) string {
if s.gatewayLocator == nil {
return ""
}
return s.gatewayLocator.PickGateway(dc)
}
// RefreshPrimaryGatewayFallbackAddresses is used to update the list of current
// fallback addresses for locating mesh gateways in the primary datacenter.
func (s *Server) RefreshPrimaryGatewayFallbackAddresses(addrs []string) {
if s.gatewayLocator != nil {
s.gatewayLocator.RefreshPrimaryGatewayFallbackAddresses(addrs)
}
}
// PrimaryGatewayFallbackAddresses returns the current set of discovered
// fallback addresses for the mesh gateways in the primary datacenter.
func (s *Server) PrimaryGatewayFallbackAddresses() []string {
if s.gatewayLocator == nil {
return nil
}
return s.gatewayLocator.PrimaryGatewayFallbackAddresses()
}
// LocalMember is used to return the local node
func (s *Server) LocalMember() serf.Member {
return s.serfLAN.LocalMember()
@ -1060,6 +1143,12 @@ func (s *Server) IsLeader() bool {
return s.raft.State() == raft.Leader
}
// LeaderLastContact returns the time of last contact by a leader.
// This only makes sense if we are currently a follower.
func (s *Server) LeaderLastContact() time.Time {
return s.raft.LastContact()
}
// KeyManagerLAN returns the LAN Serf keyring manager
func (s *Server) KeyManagerLAN() *serf.KeyManager {
return s.serfLAN.KeyManager()

View File

@ -10,6 +10,7 @@ func init() {
registerEndpoint(func(s *Server) interface{} { return NewCoordinate(s, s.logger) })
registerEndpoint(func(s *Server) interface{} { return &ConfigEntry{s} })
registerEndpoint(func(s *Server) interface{} { return &ConnectCA{srv: s, logger: s.loggers.Named(logging.Connect)} })
registerEndpoint(func(s *Server) interface{} { return &FederationState{s} })
registerEndpoint(func(s *Server) interface{} { return &DiscoveryChain{s} })
registerEndpoint(func(s *Server) interface{} { return &Health{s} })
registerEndpoint(func(s *Server) interface{} { return &Intention{s, s.loggers.Named(logging.Intentions)} })

View File

@ -7,11 +7,13 @@ import (
"strings"
"time"
"github.com/hashicorp/consul/agent/consul/wanfed"
"github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/logging"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/memberlist"
"github.com/hashicorp/raft"
"github.com/hashicorp/serf/serf"
)
@ -115,11 +117,52 @@ func (s *Server) setupSerf(conf *serf.Config, ch chan serf.Event, path string, w
}
}
if wan {
nt, err := memberlist.NewNetTransport(&memberlist.NetTransportConfig{
BindAddrs: []string{conf.MemberlistConfig.BindAddr},
BindPort: conf.MemberlistConfig.BindPort,
Logger: conf.MemberlistConfig.Logger,
})
if err != nil {
return nil, err
}
if s.config.ConnectMeshGatewayWANFederationEnabled {
mgwTransport, err := wanfed.NewTransport(
s.tlsConfigurator,
nt,
s.config.Datacenter,
s.gatewayLocator.PickGateway,
)
if err != nil {
return nil, err
}
conf.MemberlistConfig.Transport = mgwTransport
} else {
conf.MemberlistConfig.Transport = nt
}
}
// Until Consul supports this fully, we disable automatic resolution.
// When enabled, the Serf gossip may just turn off if we are the minority
// node which is rather unexpected.
conf.EnableNameConflictResolution = false
if wan && s.config.ConnectMeshGatewayWANFederationEnabled {
conf.MemberlistConfig.RequireNodeNames = true
conf.MemberlistConfig.DisableTcpPingsForNode = func(nodeName string) bool {
_, dc, err := wanfed.SplitNodeName(nodeName)
if err != nil {
return false // don't disable anything if we don't understand the node name
}
// If doing cross-dc we will be using TCP via the gateways so
// there's no need for an extra TCP request.
return s.config.Datacenter != dc
}
}
if !s.config.DevMode {
conf.SnapshotPath = filepath.Join(s.config.DataDir, path)
}
@ -319,7 +362,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,
if err := s.connPool.RPC(s.config.Datacenter, server.ShortName, server.Addr, server.Version,
"Status.Peers", server.UseTLS, &structs.DCSpecificRequest{Datacenter: s.config.Datacenter}, &peers); err != nil {
nextRetry := time.Duration((1 << attempt) * peerRetryBase)
s.logger.Error("Failed to confirm peer status for server (will retry).",

View File

@ -10,7 +10,9 @@ import (
"testing"
"time"
"github.com/google/tcpproxy"
"github.com/hashicorp/consul/agent/connect/ca"
"github.com/hashicorp/consul/ipaddr"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/metadata"
@ -60,6 +62,7 @@ func configureTLS(config *Config) {
var id int64
func uniqueNodeName(name string) string {
name = strings.ReplaceAll(name, "/", "_")
return fmt.Sprintf("%s-node-%d", name, atomic.AddInt64(&id, 1))
}
@ -543,6 +546,239 @@ func TestServer_JoinWAN_Flood(t *testing.T) {
}
}
// This is a mirror of a similar test in agent/agent_test.go
func TestServer_JoinWAN_viaMeshGateway(t *testing.T) {
t.Parallel()
gwPort := freeport.MustTake(1)
defer freeport.Return(gwPort)
gwAddr := ipaddr.FormatAddressPort("127.0.0.1", gwPort[0])
dir1, s1 := testServerWithConfig(t, func(c *Config) {
c.Domain = "consul"
c.NodeName = "bob"
c.Datacenter = "dc1"
c.PrimaryDatacenter = "dc1"
c.Bootstrap = true
// tls
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Bob.crt"
c.KeyFile = "../../test/hostname/Bob.key"
c.VerifyIncoming = true
c.VerifyOutgoing = true
c.VerifyServerHostname = true
// wanfed
c.ConnectMeshGatewayWANFederationEnabled = true
})
defer os.RemoveAll(dir1)
defer s1.Shutdown()
dir2, s2 := testServerWithConfig(t, func(c *Config) {
c.Domain = "consul"
c.NodeName = "betty"
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc1"
c.Bootstrap = true
// tls
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Betty.crt"
c.KeyFile = "../../test/hostname/Betty.key"
c.VerifyIncoming = true
c.VerifyOutgoing = true
c.VerifyServerHostname = true
// wanfed
c.ConnectMeshGatewayWANFederationEnabled = true
})
defer os.RemoveAll(dir2)
defer s2.Shutdown()
dir3, s3 := testServerWithConfig(t, func(c *Config) {
c.Domain = "consul"
c.NodeName = "bonnie"
c.Datacenter = "dc3"
c.PrimaryDatacenter = "dc1"
c.Bootstrap = true
// tls
c.CAFile = "../../test/hostname/CertAuth.crt"
c.CertFile = "../../test/hostname/Bonnie.crt"
c.KeyFile = "../../test/hostname/Bonnie.key"
c.VerifyIncoming = true
c.VerifyOutgoing = true
c.VerifyServerHostname = true
// wanfed
c.ConnectMeshGatewayWANFederationEnabled = true
})
defer os.RemoveAll(dir3)
defer s3.Shutdown()
// We'll use the same gateway for all datacenters since it doesn't care.
var p tcpproxy.Proxy
p.AddSNIRoute(gwAddr, "bob.server.dc1.consul", tcpproxy.To(s1.config.RPCAddr.String()))
p.AddSNIRoute(gwAddr, "betty.server.dc2.consul", tcpproxy.To(s2.config.RPCAddr.String()))
p.AddSNIRoute(gwAddr, "bonnie.server.dc3.consul", tcpproxy.To(s3.config.RPCAddr.String()))
p.AddStopACMESearch(gwAddr)
require.NoError(t, p.Start())
defer func() {
p.Close()
p.Wait()
}()
t.Logf("routing %s => %s", "bob.server.dc1.consul", s1.config.RPCAddr.String())
t.Logf("routing %s => %s", "betty.server.dc2.consul", s2.config.RPCAddr.String())
t.Logf("routing %s => %s", "bonnie.server.dc3.consul", s3.config.RPCAddr.String())
// Register this into the catalog in dc1.
{
arg := structs.RegisterRequest{
Datacenter: "dc1",
Node: "bob",
Address: "127.0.0.1",
Service: &structs.NodeService{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Service: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
},
}
var out struct{}
require.NoError(t, s1.RPC("Catalog.Register", &arg, &out))
}
// Wait for it to make it into the gateway locator.
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, s1.gatewayLocator.PickGateway("dc1"))
})
// Seed the secondaries with the address of the primary and wait for that to
// be in their locators.
s2.RefreshPrimaryGatewayFallbackAddresses([]string{gwAddr})
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, s2.gatewayLocator.PickGateway("dc1"))
})
s3.RefreshPrimaryGatewayFallbackAddresses([]string{gwAddr})
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, s3.gatewayLocator.PickGateway("dc1"))
})
// Try to join from secondary to primary. We can't use joinWAN() because we
// are simulating proper bootstrapping and if ACLs were on we would have to
// delay gateway registration in the secondary until after one directional
// join. So this way we explicitly join secondary-to-primary as a standalone
// operation and follow it up later with a full join.
_, err := s2.JoinWAN([]string{joinAddrWAN(s1)})
require.NoError(t, err)
retry.Run(t, func(r *retry.R) {
if got, want := len(s2.WANMembers()), 2; got != want {
r.Fatalf("got %d s2 WAN members want %d", got, want)
}
})
_, err = s3.JoinWAN([]string{joinAddrWAN(s1)})
require.NoError(t, err)
retry.Run(t, func(r *retry.R) {
if got, want := len(s3.WANMembers()), 3; got != want {
r.Fatalf("got %d s3 WAN members want %d", got, want)
}
})
// Now we can register this into the catalog in dc2 and dc3.
{
arg := structs.RegisterRequest{
Datacenter: "dc2",
Node: "betty",
Address: "127.0.0.1",
Service: &structs.NodeService{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Service: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
},
}
var out struct{}
require.NoError(t, s2.RPC("Catalog.Register", &arg, &out))
}
{
arg := structs.RegisterRequest{
Datacenter: "dc3",
Node: "bonnie",
Address: "127.0.0.1",
Service: &structs.NodeService{
Kind: structs.ServiceKindMeshGateway,
ID: "mesh-gateway",
Service: "mesh-gateway",
Meta: map[string]string{structs.MetaWANFederationKey: "1"},
Port: gwPort[0],
},
}
var out struct{}
require.NoError(t, s3.RPC("Catalog.Register", &arg, &out))
}
// Wait for it to make it into the gateway locator in dc2 and then for
// AE to carry it back to the primary
retry.Run(t, func(r *retry.R) {
require.NotEmpty(r, s3.gatewayLocator.PickGateway("dc2"))
require.NotEmpty(r, s2.gatewayLocator.PickGateway("dc2"))
require.NotEmpty(r, s1.gatewayLocator.PickGateway("dc2"))
require.NotEmpty(r, s3.gatewayLocator.PickGateway("dc3"))
require.NotEmpty(r, s2.gatewayLocator.PickGateway("dc3"))
require.NotEmpty(r, s1.gatewayLocator.PickGateway("dc3"))
})
// Try to join again using the standard verification method now that
// all of the plumbing is in place.
joinWAN(t, s2, s1)
retry.Run(t, func(r *retry.R) {
if got, want := len(s1.WANMembers()), 3; got != want {
r.Fatalf("got %d s1 WAN members want %d", got, want)
}
if got, want := len(s2.WANMembers()), 3; got != want {
r.Fatalf("got %d s2 WAN members want %d", got, want)
}
})
// Check the router has all of them
retry.Run(t, func(r *retry.R) {
if got, want := len(s1.router.GetDatacenters()), 3; got != want {
r.Fatalf("got %d routes want %d", got, want)
}
if got, want := len(s2.router.GetDatacenters()), 3; got != want {
r.Fatalf("got %d datacenters want %d", got, want)
}
if got, want := len(s3.router.GetDatacenters()), 3; got != want {
r.Fatalf("got %d datacenters want %d", got, want)
}
})
// Ensure we can do some trivial RPC in all directions.
servers := map[string]*Server{"dc1": s1, "dc2": s2, "dc3": s3}
names := map[string]string{"dc1": "bob", "dc2": "betty", "dc3": "bonnie"}
for _, srcDC := range []string{"dc1", "dc2", "dc3"} {
srv := servers[srcDC]
for _, dstDC := range []string{"dc1", "dc2", "dc3"} {
if srcDC == dstDC {
continue
}
t.Run(srcDC+" to "+dstDC, func(t *testing.T) {
arg := structs.DCSpecificRequest{
Datacenter: dstDC,
}
var out structs.IndexedNodes
require.NoError(t, srv.RPC("Catalog.ListNodes", &arg, &out))
require.Len(t, out.Nodes, 1)
node := out.Nodes[0]
require.Equal(t, dstDC, node.Datacenter)
require.Equal(t, names[dstDC], node.Node)
})
}
}
}
func TestServer_JoinSeparateLanAndWanAddresses(t *testing.T) {
t.Parallel()
dir1, s1 := testServerWithConfig(t, func(c *Config) {
@ -999,7 +1235,7 @@ func testVerifyRPC(s1, s2 *Server, t *testing.T) (bool, error) {
if leader == nil {
t.Fatal("no leader")
}
return s2.connPool.Ping(leader.Datacenter, leader.Addr, leader.Version, leader.UseTLS)
return s2.connPool.Ping(leader.Datacenter, leader.ShortName, leader.Addr, leader.Version, leader.UseTLS)
}
func TestServer_TLSToNoTLS(t *testing.T) {

View File

@ -37,7 +37,7 @@ func (s *Server) dispatchSnapshotRequest(args *structs.SnapshotRequest, in io.Re
return nil, structs.ErrNoDCPath
}
snap, err := SnapshotRPC(s.connPool, dc, server.Addr, server.UseTLS, args, in, reply)
snap, err := SnapshotRPC(s.connPool, dc, server.ShortName, server.Addr, server.UseTLS, args, in, reply)
if err != nil {
manager.NotifyFailedServer(server)
return nil, err
@ -52,7 +52,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, server.UseTLS, args, in, reply)
return SnapshotRPC(s.connPool, args.Datacenter, server.ShortName, server.Addr, server.UseTLS, args, in, reply)
}
}
@ -189,10 +189,19 @@ 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(connPool *pool.ConnPool, dc string, addr net.Addr, useTLS bool,
args *structs.SnapshotRequest, in io.Reader, reply *structs.SnapshotResponse) (io.ReadCloser, error) {
conn, hc, err := connPool.DialTimeout(dc, addr, 10*time.Second, useTLS)
func SnapshotRPC(
connPool *pool.ConnPool,
dc string,
nodeName string,
addr net.Addr,
useTLS bool,
args *structs.SnapshotRequest,
in io.Reader,
reply *structs.SnapshotResponse,
) (io.ReadCloser, error) {
// Write the snapshot RPC byte to set the mode, then perform the
// request.
conn, hc, err := connPool.DialTimeout(dc, nodeName, addr, 10*time.Second, useTLS, pool.RPCSnapshot)
if err != nil {
return nil, err
}
@ -206,12 +215,6 @@ func SnapshotRPC(connPool *pool.ConnPool, dc string, addr net.Addr, useTLS bool,
}
}()
// Write the snapshot RPC byte to set the mode, then perform the
// request.
if _, err := conn.Write([]byte{byte(pool.RPCSnapshot)}); err != nil {
return nil, fmt.Errorf("failed to write stream type: %v", err)
}
// Push the header encoded as msgpack, then stream the input.
enc := codec.NewEncoder(conn, structs.MsgpackHandle)
if err := enc.Encode(&args); err != nil {

View File

@ -46,7 +46,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, false,
snap, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.NodeName, s.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -121,7 +121,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, false,
restore, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.NodeName, s.config.RPCAddr, false,
&args, snap, &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -196,7 +196,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, false,
snap, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.NodeName, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -229,7 +229,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, false,
restore, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.NodeName, s1.config.RPCAddr, false,
&args, snap, &reply)
if err != nil {
t.Fatalf("err: %v", err)
@ -268,7 +268,7 @@ func TestSnapshot_ACLDeny(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.NodeName, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
@ -282,7 +282,7 @@ func TestSnapshot_ACLDeny(t *testing.T) {
Op: structs.SnapshotRestore,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.RPCAddr, false,
_, err := SnapshotRPC(s1.connPool, s1.config.Datacenter, s1.config.NodeName, s1.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if !acl.IsErrPermissionDenied(err) {
t.Fatalf("err: %v", err)
@ -391,7 +391,7 @@ func TestSnapshot_AllowStale(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.NodeName, s.config.RPCAddr, false,
&args, bytes.NewReader([]byte("")), &reply)
if err == nil || !strings.Contains(err.Error(), structs.ErrNoLeader.Error()) {
t.Fatalf("err: %v", err)
@ -408,7 +408,7 @@ func TestSnapshot_AllowStale(t *testing.T) {
Op: structs.SnapshotSave,
}
var reply structs.SnapshotResponse
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.RPCAddr, false,
_, err := SnapshotRPC(s.connPool, s.config.Datacenter, s.config.NodeName, 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

@ -671,7 +671,7 @@ func (s *Store) ensureServiceTxn(tx *memdb.Txn, idx uint64, node string, svc *st
return fmt.Errorf("failed service lookup: %s", err)
}
if err = structs.ValidateMetadata(svc.Meta, false); err != nil {
if err = structs.ValidateServiceMetadata(svc.Kind, svc.Meta, false); err != nil {
return fmt.Errorf("Invalid Service Meta for node %s and serviceID %s: %v", node, svc.ID, err)
}
// Create the service node entry and populate the indexes. Note that

View File

@ -0,0 +1,230 @@
package state
import (
"fmt"
"github.com/hashicorp/consul/agent/structs"
memdb "github.com/hashicorp/go-memdb"
)
const federationStateTableName = "federation-states"
func federationStateTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: federationStateTableName,
Indexes: map[string]*memdb.IndexSchema{
"id": &memdb.IndexSchema{
Name: "id",
AllowMissing: false,
Unique: true,
Indexer: &memdb.StringFieldIndex{
Field: "Datacenter",
Lowercase: true,
},
},
},
}
}
func init() {
registerSchema(federationStateTableSchema)
}
// FederationStates is used to pull all the federation states for the snapshot.
func (s *Snapshot) FederationStates() ([]*structs.FederationState, error) {
configs, err := s.tx.Get(federationStateTableName, "id")
if err != nil {
return nil, err
}
var ret []*structs.FederationState
for wrapped := configs.Next(); wrapped != nil; wrapped = configs.Next() {
ret = append(ret, wrapped.(*structs.FederationState))
}
return ret, nil
}
// FederationState is used when restoring from a snapshot.
func (s *Restore) FederationState(g *structs.FederationState) error {
// Insert
if err := s.tx.Insert(federationStateTableName, g); err != nil {
return fmt.Errorf("failed restoring federation state object: %s", err)
}
if err := indexUpdateMaxTxn(s.tx, g.ModifyIndex, federationStateTableName); err != nil {
return fmt.Errorf("failed updating index: %s", err)
}
return nil
}
func (s *Store) FederationStateBatchSet(idx uint64, configs structs.FederationStates) error {
tx := s.db.Txn(true)
defer tx.Abort()
for _, config := range configs {
if err := s.federationStateSetTxn(tx, idx, config); err != nil {
return err
}
}
tx.Commit()
return nil
}
// FederationStateSet is called to do an upsert of a given federation state.
func (s *Store) FederationStateSet(idx uint64, config *structs.FederationState) error {
tx := s.db.Txn(true)
defer tx.Abort()
if err := s.federationStateSetTxn(tx, idx, config); err != nil {
return err
}
tx.Commit()
return nil
}
// federationStateSetTxn upserts a federation state inside of a transaction.
func (s *Store) federationStateSetTxn(tx *memdb.Txn, idx uint64, config *structs.FederationState) error {
if config.Datacenter == "" {
return fmt.Errorf("missing datacenter on federation state")
}
// Check for existing.
var existing *structs.FederationState
existingRaw, err := tx.First(federationStateTableName, "id", config.Datacenter)
if err != nil {
return fmt.Errorf("failed federation state lookup: %s", err)
}
if existingRaw != nil {
existing = existingRaw.(*structs.FederationState)
}
// Set the indexes
if existing != nil {
config.CreateIndex = existing.CreateIndex
config.ModifyIndex = idx
} else {
config.CreateIndex = idx
config.ModifyIndex = idx
}
if config.PrimaryModifyIndex == 0 {
// Since replication ordinarily would set this value for us, we can
// assume this is a write to the primary datacenter's federation state
// so we can just duplicate the new modify index.
config.PrimaryModifyIndex = idx
}
// Insert the federation state and update the index
if err := tx.Insert(federationStateTableName, config); err != nil {
return fmt.Errorf("failed inserting federation state: %s", err)
}
if err := tx.Insert("index", &IndexEntry{federationStateTableName, idx}); err != nil {
return fmt.Errorf("failed updating index: %v", err)
}
return nil
}
// FederationStateGet is called to get a federation state.
func (s *Store) FederationStateGet(ws memdb.WatchSet, datacenter string) (uint64, *structs.FederationState, error) {
tx := s.db.Txn(false)
defer tx.Abort()
return s.federationStateGetTxn(tx, ws, datacenter)
}
func (s *Store) federationStateGetTxn(tx *memdb.Txn, ws memdb.WatchSet, datacenter string) (uint64, *structs.FederationState, error) {
// Get the index
idx := maxIndexTxn(tx, federationStateTableName)
// Get the existing contents.
watchCh, existing, err := tx.FirstWatch(federationStateTableName, "id", datacenter)
if err != nil {
return 0, nil, fmt.Errorf("failed federation state lookup: %s", err)
}
ws.Add(watchCh)
if existing == nil {
return idx, nil, nil
}
config, ok := existing.(*structs.FederationState)
if !ok {
return 0, nil, fmt.Errorf("federation state %q is an invalid type: %T", datacenter, config)
}
return idx, config, nil
}
// FederationStateList is called to get all federation state objects.
func (s *Store) FederationStateList(ws memdb.WatchSet) (uint64, []*structs.FederationState, error) {
tx := s.db.Txn(false)
defer tx.Abort()
return s.federationStateListTxn(tx, ws)
}
func (s *Store) federationStateListTxn(tx *memdb.Txn, ws memdb.WatchSet) (uint64, []*structs.FederationState, error) {
// Get the index
idx := maxIndexTxn(tx, federationStateTableName)
iter, err := tx.Get(federationStateTableName, "id")
if err != nil {
return 0, nil, fmt.Errorf("failed federation state lookup: %s", err)
}
ws.Add(iter.WatchCh())
var results []*structs.FederationState
for v := iter.Next(); v != nil; v = iter.Next() {
results = append(results, v.(*structs.FederationState))
}
return idx, results, nil
}
func (s *Store) FederationStateDelete(idx uint64, datacenter string) error {
tx := s.db.Txn(true)
defer tx.Abort()
if err := s.federationStateDeleteTxn(tx, idx, datacenter); err != nil {
return err
}
tx.Commit()
return nil
}
func (s *Store) FederationStateBatchDelete(idx uint64, datacenters []string) error {
tx := s.db.Txn(true)
defer tx.Abort()
for _, datacenter := range datacenters {
if err := s.federationStateDeleteTxn(tx, idx, datacenter); err != nil {
return err
}
}
tx.Commit()
return nil
}
func (s *Store) federationStateDeleteTxn(tx *memdb.Txn, idx uint64, datacenter string) error {
// Try to retrieve the existing federation state.
existing, err := tx.First(federationStateTableName, "id", datacenter)
if err != nil {
return fmt.Errorf("failed federation state lookup: %s", err)
}
if existing == nil {
return nil
}
// Delete the federation state from the DB and update the index.
if err := tx.Delete(federationStateTableName, existing); err != nil {
return fmt.Errorf("failed removing federation state: %s", err)
}
if err := tx.Insert("index", &IndexEntry{federationStateTableName, idx}); err != nil {
return fmt.Errorf("failed updating index: %s", err)
}
return nil
}

View File

@ -43,7 +43,7 @@ func NewStatsFetcher(logger hclog.Logger, pool *pool.ConnPool, datacenter string
func (f *StatsFetcher) fetch(server *metadata.Server, replyCh chan *autopilot.ServerStats) {
var args struct{}
var reply autopilot.ServerStats
err := f.pool.RPC(f.datacenter, server.Addr, server.Version, "Status.RaftStats", server.UseTLS, &args, &reply)
err := f.pool.RPC(f.datacenter, server.ShortName, server.Addr, server.Version, "Status.RaftStats", server.UseTLS, &args, &reply)
if err != nil {
f.logger.Warn("error getting server health from server",
"server", server.Name,

View File

@ -29,7 +29,7 @@ func rpcClient(t *testing.T, s *Server) rpc.ClientCodec {
func insecureRPCClient(s *Server, c tlsutil.Config) (rpc.ClientCodec, error) {
addr := s.config.RPCAdvertise
configurator, err := tlsutil.NewConfigurator(c, nil)
configurator, err := tlsutil.NewConfigurator(c, s.logger)
if err != nil {
return nil, err
}
@ -37,7 +37,17 @@ func insecureRPCClient(s *Server, c tlsutil.Config) (rpc.ClientCodec, error) {
if wrapper == nil {
return nil, err
}
conn, _, err := pool.DialTimeoutWithRPCType(s.config.Datacenter, addr, nil, time.Second, true, wrapper, pool.RPCTLSInsecure)
conn, _, err := pool.DialTimeoutWithRPCTypeDirectly(
s.config.Datacenter,
s.config.NodeName,
addr,
nil,
time.Second,
true,
wrapper,
pool.RPCTLSInsecure,
pool.RPCTLSInsecure,
)
if err != nil {
return nil, err
}

255
agent/consul/wanfed/pool.go Normal file
View File

@ -0,0 +1,255 @@
package wanfed
import (
"fmt"
"net"
"sync"
"time"
)
// connPool pools idle negotiated ALPN_WANGossipPacket flavored connections to
// remote servers. Idle connections only remain pooled for up to maxTime after
// they were last acquired.
type connPool struct {
// maxTime is the maximum time to keep a connection open.
maxTime time.Duration
// mu protects pool and shutdown
mu sync.Mutex
pool map[string][]*conn
shutdown bool
shutdownCh chan struct{}
reapWg sync.WaitGroup
}
func newConnPool(maxTime time.Duration) (*connPool, error) {
if maxTime == 0 {
return nil, fmt.Errorf("wanfed: conn pool needs a max time configured")
}
p := &connPool{
maxTime: maxTime,
pool: make(map[string][]*conn),
shutdownCh: make(chan struct{}),
}
p.reapWg.Add(1)
go p.reap()
return p, nil
}
func (p *connPool) Close() error {
p.mu.Lock()
defer p.mu.Unlock()
if p.shutdown {
return nil
}
for _, conns := range p.pool {
for _, conn := range conns {
conn.Close()
}
}
p.pool = nil
p.shutdown = true
close(p.shutdownCh)
p.reapWg.Wait()
return nil
}
// AcquireOrDial either removes an idle connection from the pool or
// estabilishes a new one using the provided dialer function.
func (p *connPool) AcquireOrDial(key string, dialer func() (net.Conn, error)) (*conn, error) {
c, err := p.maybeAcquire(key)
if err != nil {
return nil, err
}
if c != nil {
c.markForUse()
return c, nil
}
nc, err := dialer()
if err != nil {
return nil, err
}
c = &conn{
key: key,
pool: p,
Conn: nc,
}
c.markForUse()
return c, nil
}
var errPoolClosed = fmt.Errorf("wanfed: connection pool is closed")
// maybeAcquire removes an idle connection from the pool if possible otherwise
// returns nil indicating there were no idle connections ready. It is the
// caller's responsibility to open a new connection if that is desired.
func (p *connPool) maybeAcquire(key string) (*conn, error) {
p.mu.Lock()
defer p.mu.Unlock()
if p.shutdown {
return nil, errPoolClosed
}
conns, ok := p.pool[key]
if !ok {
return nil, nil
}
switch len(conns) {
case 0:
delete(p.pool, key) // stray cleanup
return nil, nil
case 1:
c := conns[0]
delete(p.pool, key)
return c, nil
default:
sz := len(conns)
remaining, last := conns[0:sz-1], conns[sz-1]
p.pool[key] = remaining
return last, nil
}
}
// returnConn puts the connection back into the idle pool for reuse.
func (p *connPool) returnConn(c *conn) error {
p.mu.Lock()
defer p.mu.Unlock()
if p.shutdown {
return c.Conn.Close() // actual shutdown
}
p.pool[c.key] = append(p.pool[c.key], c)
return nil
}
// reap periodically scans the idle pool for connections that have not been
// used recently and closes them.
func (p *connPool) reap() {
defer p.reapWg.Done()
for {
select {
case <-p.shutdownCh:
return
case <-time.After(time.Second):
}
p.reapOnce()
}
}
func (p *connPool) reapOnce() {
p.mu.Lock()
defer p.mu.Unlock()
if p.shutdown {
return
}
now := time.Now()
var removedKeys []string
for key, conns := range p.pool {
if len(conns) == 0 {
removedKeys = append(removedKeys, key) // cleanup
continue
}
var retain []*conn
for _, c := range conns {
// Skip recently used connections
if now.Sub(c.lastUsed) < p.maxTime {
retain = append(retain, c)
} else {
c.Conn.Close()
}
}
if len(retain) == len(conns) {
continue // no change
} else if len(retain) == 0 {
removedKeys = append(removedKeys, key)
continue
}
p.pool[key] = retain
}
for _, key := range removedKeys {
delete(p.pool, key)
}
}
type conn struct {
key string
mu sync.Mutex
lastUsed time.Time
failed bool
closed bool
pool *connPool
net.Conn
}
func (c *conn) ReturnOrClose() error {
c.mu.Lock()
closed := c.closed
failed := c.failed
if failed {
c.closed = true
}
c.mu.Unlock()
if closed {
return nil
}
if failed {
return c.Conn.Close()
}
return c.pool.returnConn(c)
}
func (c *conn) Close() error {
c.mu.Lock()
closed := c.closed
c.closed = true
c.mu.Unlock()
if closed {
return nil
}
return c.Conn.Close()
}
func (c *conn) markForUse() {
c.mu.Lock()
c.lastUsed = time.Now()
c.failed = false
c.mu.Unlock()
}
func (c *conn) MarkFailed() {
c.mu.Lock()
c.failed = true
c.mu.Unlock()
}

View File

@ -0,0 +1,179 @@
package wanfed
import (
"encoding/binary"
"errors"
"fmt"
"net"
"strings"
"time"
"github.com/hashicorp/consul/agent/pool"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/memberlist"
)
const (
// GossipPacketMaxIdleTime controls how long we keep an idle connection
// open to a server.
//
// Conceptually similar to: agent/consul/server.go:serverRPCCache
GossipPacketMaxIdleTime = 2 * time.Minute
// GossipPacketMaxByteSize is the maximum allowed size of a packet
// forwarded via wanfed. This is 4MB which should be way bigger than serf
// or memberlist allow practically so it should never be hit in practice.
GossipPacketMaxByteSize = 4 * 1024 * 1024
)
type MeshGatewayResolver func(datacenter string) string
func NewTransport(
tlsConfigurator *tlsutil.Configurator,
transport memberlist.NodeAwareTransport,
datacenter string,
gwResolver MeshGatewayResolver,
) (*Transport, error) {
if tlsConfigurator == nil {
return nil, errors.New("wanfed: tlsConfigurator is nil")
}
if gwResolver == nil {
return nil, errors.New("wanfed: gwResolver is nil")
}
cp, err := newConnPool(GossipPacketMaxIdleTime)
if err != nil {
return nil, err
}
t := &Transport{
NodeAwareTransport: transport,
tlsConfigurator: tlsConfigurator,
datacenter: datacenter,
gwResolver: gwResolver,
pool: cp,
}
return t, nil
}
type Transport struct {
memberlist.NodeAwareTransport
tlsConfigurator *tlsutil.Configurator
datacenter string
gwResolver MeshGatewayResolver
pool *connPool
}
var _ memberlist.NodeAwareTransport = (*Transport)(nil)
// Shutdown implements memberlist.Transport.
func (t *Transport) Shutdown() error {
err1 := t.pool.Close()
err2 := t.NodeAwareTransport.Shutdown()
if err2 != nil {
// the more important error is err2
return err2
}
if err1 != nil {
return err1
}
return nil
}
// WriteToAddress implements memberlist.NodeAwareTransport.
func (t *Transport) WriteToAddress(b []byte, addr memberlist.Address) (time.Time, error) {
node, dc, err := SplitNodeName(addr.Name)
if err != nil {
return time.Time{}, err
}
if dc != t.datacenter {
gwAddr := t.gwResolver(dc)
if gwAddr == "" {
return time.Time{}, structs.ErrDCNotAvailable
}
dialFunc := func() (net.Conn, error) {
return t.dial(dc, node, pool.ALPN_WANGossipPacket, gwAddr)
}
conn, err := t.pool.AcquireOrDial(addr.Name, dialFunc)
if err != nil {
return time.Time{}, err
}
defer conn.ReturnOrClose()
// Send the length first.
if err := binary.Write(conn, binary.BigEndian, uint32(len(b))); err != nil {
conn.MarkFailed()
return time.Time{}, err
}
if _, err = conn.Write(b); err != nil {
conn.MarkFailed()
return time.Time{}, err
}
return time.Now(), nil
}
return t.NodeAwareTransport.WriteToAddress(b, addr)
}
// DialAddressTimeout implements memberlist.NodeAwareTransport.
func (t *Transport) DialAddressTimeout(addr memberlist.Address, timeout time.Duration) (net.Conn, error) {
node, dc, err := SplitNodeName(addr.Name)
if err != nil {
return nil, err
}
if dc != t.datacenter {
gwAddr := t.gwResolver(dc)
if gwAddr == "" {
return nil, structs.ErrDCNotAvailable
}
return t.dial(dc, node, pool.ALPN_WANGossipStream, gwAddr)
}
return t.NodeAwareTransport.DialAddressTimeout(addr, timeout)
}
// NOTE: There is a close mirror of this method in agent/pool/pool.go:DialTimeoutWithRPCType
func (t *Transport) dial(dc, nodeName, nextProto, addr string) (net.Conn, error) {
wrapper := t.tlsConfigurator.OutgoingALPNRPCWrapper()
if wrapper == nil {
return nil, fmt.Errorf("wanfed: cannot dial via a mesh gateway when outgoing TLS is disabled")
}
dialer := &net.Dialer{Timeout: 10 * time.Second}
rawConn, err := dialer.Dial("tcp", addr)
if err != nil {
return nil, err
}
if tcp, ok := rawConn.(*net.TCPConn); ok {
_ = tcp.SetKeepAlive(true)
_ = tcp.SetNoDelay(true)
}
tlsConn, err := wrapper(dc, nodeName, nextProto, rawConn)
if err != nil {
return nil, err
}
return tlsConn, nil
}
// SplitNodeName splits a node name as it would be represented in
// serf/memberlist in the WAN pool of the form "<short-node-name>.<datacenter>"
// like "nyc-web42.dc5" => "nyc-web42" & "dc5"
func SplitNodeName(nodeName string) (shortName, dc string, err error) {
parts := strings.Split(nodeName, ".")
if len(parts) != 2 {
return "", "", fmt.Errorf("node name does not encode a datacenter: %s", nodeName)
}
return parts[0], parts[1], nil
}

View File

@ -0,0 +1,43 @@
package wanfed
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestSplitNodeName(t *testing.T) {
type testcase struct {
nodeName string
expectShortName string
expectDC string
expectErr bool
}
cases := []testcase{
// bad
{nodeName: "", expectErr: true},
{nodeName: "foo", expectErr: true},
{nodeName: "foo.bar.baz", expectErr: true},
// good
{nodeName: "foo.bar", expectShortName: "foo", expectDC: "bar"},
// weird
{nodeName: ".bar", expectShortName: "", expectDC: "bar"},
}
for _, tc := range cases {
tc := tc
t.Run(tc.nodeName, func(t *testing.T) {
gotShortName, gotDC, gotErr := SplitNodeName(tc.nodeName)
if tc.expectErr {
require.Error(t, gotErr)
require.Empty(t, gotShortName)
require.Empty(t, gotDC)
} else {
require.NoError(t, gotErr)
require.Equal(t, tc.expectShortName, gotShortName)
require.Equal(t, tc.expectDC, gotDC)
}
})
}
}

View File

@ -0,0 +1,91 @@
package agent
import (
"net/http"
"strings"
"github.com/hashicorp/consul/agent/structs"
)
// GET /v1/internal/federation-state/<datacenter>
func (s *HTTPServer) FederationStateGet(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
datacenterName := strings.TrimPrefix(req.URL.Path, "/v1/internal/federation-state/")
if datacenterName == "" {
return nil, BadRequestError{Reason: "Missing datacenter name"}
}
args := structs.FederationStateQuery{
Datacenter: datacenterName,
}
if done := s.parse(resp, req, &args.TargetDatacenter, &args.QueryOptions); done {
return nil, nil
}
var out structs.FederationStateResponse
defer setMeta(resp, &out.QueryMeta)
if err := s.agent.RPC("FederationState.Get", &args, &out); err != nil {
return nil, err
}
if out.State == nil {
resp.WriteHeader(http.StatusNotFound)
return nil, nil
}
return out, nil
}
// GET /v1/internal/federation-states
func (s *HTTPServer) FederationStateList(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
var args structs.DCSpecificRequest
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
var out structs.IndexedFederationStates
defer setMeta(resp, &out.QueryMeta)
if err := s.agent.RPC("FederationState.List", &args, &out); err != nil {
return nil, err
}
// make sure we return an array and not nil
if out.States == nil {
out.States = make(structs.FederationStates, 0)
}
return out.States, nil
}
// GET /v1/internal/federation-states/mesh-gateways
func (s *HTTPServer) FederationStateListMeshGateways(resp http.ResponseWriter, req *http.Request) (interface{}, error) {
var args structs.DCSpecificRequest
if done := s.parse(resp, req, &args.Datacenter, &args.QueryOptions); done {
return nil, nil
}
if args.Datacenter == "" {
args.Datacenter = s.agent.config.Datacenter
}
var out structs.DatacenterIndexedCheckServiceNodes
defer setMeta(resp, &out.QueryMeta)
if err := s.agent.RPC("FederationState.ListMeshGateways", &args, &out); err != nil {
return nil, err
}
// make sure we return a arrays and not nils
if out.DatacenterNodes == nil {
out.DatacenterNodes = make(map[string]structs.CheckServiceNodes)
}
for dc, nodes := range out.DatacenterNodes {
if nodes == nil {
out.DatacenterNodes[dc] = make(structs.CheckServiceNodes, 0)
}
}
return out.DatacenterNodes, nil
}

View File

@ -79,6 +79,9 @@ func init() {
registerEndpoint("/v1/coordinate/nodes", []string{"GET"}, (*HTTPServer).CoordinateNodes)
registerEndpoint("/v1/coordinate/node/", []string{"GET"}, (*HTTPServer).CoordinateNode)
registerEndpoint("/v1/coordinate/update", []string{"PUT"}, (*HTTPServer).CoordinateUpdate)
registerEndpoint("/v1/internal/federation-states", []string{"GET"}, (*HTTPServer).FederationStateList)
registerEndpoint("/v1/internal/federation-states/mesh-gateways", []string{"GET"}, (*HTTPServer).FederationStateListMeshGateways)
registerEndpoint("/v1/internal/federation-state/", []string{"GET"}, (*HTTPServer).FederationStateGet)
registerEndpoint("/v1/discovery-chain/", []string{"GET", "POST"}, (*HTTPServer).DiscoveryChainRead)
registerEndpoint("/v1/event/fire/", []string{"PUT"}, (*HTTPServer).EventFire)
registerEndpoint("/v1/event/list", []string{"GET"}, (*HTTPServer).EventList)

View File

@ -24,7 +24,8 @@ func (k *Key) Equal(x *Key) bool {
// Server is used to return details of a consul server
type Server struct {
Name string
Name string // <node>.<dc>
ShortName string // <node>
ID string
Datacenter string
Segment string
@ -165,6 +166,7 @@ func IsConsulServer(m serf.Member) (bool, *Server) {
parts := &Server{
Name: m.Name,
ShortName: strings.TrimSuffix(m.Name, "."+datacenter),
ID: m.Tags["id"],
Datacenter: datacenter,
Segment: segment,

View File

@ -2,6 +2,29 @@ package pool
type RPCType byte
func (t RPCType) ALPNString() string {
switch t {
case RPCConsul:
return ALPN_RPCConsul
case RPCRaft:
return ALPN_RPCRaft
case RPCMultiplex:
return "" // unsupported
case RPCTLS:
return "" // unsupported
case RPCMultiplexV2:
return ALPN_RPCMultiplexV2
case RPCSnapshot:
return ALPN_RPCSnapshot
case RPCGossip:
return ALPN_RPCGossip
case RPCTLSInsecure:
return "" // unsupported
default:
return "" // unsupported
}
}
const (
// keep numbers unique.
RPCConsul RPCType = 0
@ -13,18 +36,47 @@ const (
RPCGossip = 6
// RPCTLSInsecure is used to flag RPC calls that require verify
// incoming to be disabled, even when it is turned on in the
// configuration. At the time of writing there is only AutoEncryt.Sign
// configuration. At the time of writing there is only AutoEncrypt.Sign
// that is supported and it might be the only one there
// ever is.
RPCTLSInsecure = 7
// NOTE: Currently we use values between 0 and 7 for the different
// "protocols" that we may ride over our "rpc" port. We had an idea of
// using TLS + ALPN for negotiating the protocol instead of our own
// bytes as it could provide other benefits. Currently our 0-7 values
// are mutually exclusive with any valid first byte of a TLS header
// The first TLS header byte will content a TLS content type and the
// values 0-19 are all explicitly unassigned and marked as
// requiring coordination. RFC 7983 does the marking and goes into
// some details about multiplexing connections and identifying TLS.
// RPCMaxTypeValue is the maximum rpc type byte value currently used for
// the various protocols riding over our "rpc" port.
//
// Currently our 0-7 values are mutually exclusive with any valid first
// byte of a TLS header. The first TLS header byte will begin with a TLS
// content type and the values 0-19 are all explicitly unassigned and
// marked as requiring coordination. RFC 7983 does the marking and goes
// into some details about multiplexing connections and identifying TLS.
//
// We use this value to determine if the incoming request is actual real
// native TLS (where we can demultiplex based on ALPN protocol) or our
// older type-byte system when new connections are established.
//
// NOTE: if you add new RPCTypes beyond this value, you must similarly bump
// this value.
RPCMaxTypeValue = 7
)
const (
// regular old rpc (note there is no equivalent of RPCMultiplex, RPCTLS, or RPCTLSInsecure)
ALPN_RPCConsul = "consul/rpc-single" // RPCConsul
ALPN_RPCRaft = "consul/raft" // RPCRaft
ALPN_RPCMultiplexV2 = "consul/rpc-multi" // RPCMultiplexV2
ALPN_RPCSnapshot = "consul/rpc-snapshot" // RPCSnapshot
ALPN_RPCGossip = "consul/rpc-gossip" // RPCGossip
// wan federation additions
ALPN_WANGossipPacket = "consul/wan-gossip/packet"
ALPN_WANGossipStream = "consul/wan-gossip/stream"
)
var RPCNextProtos = []string{
ALPN_RPCConsul,
ALPN_RPCRaft,
ALPN_RPCMultiplexV2,
ALPN_RPCSnapshot,
ALPN_RPCGossip,
ALPN_WANGossipPacket,
ALPN_WANGossipStream,
}

49
agent/pool/peek.go Normal file
View File

@ -0,0 +1,49 @@
package pool
import (
"bufio"
"net"
)
// PeekForTLS will read the first byte on the conn to determine if the client
// request is a TLS connection request or a consul-specific framed rpc request.
//
// This function does not close the conn on an error.
//
// The returned conn has the initial read buffered internally for the purposes
// of not consuming the first byte. After that buffer is drained the conn is a
// pass through to the original conn.
//
// The TLS record layer governs the very first byte. The available options start
// at 20 as per:
//
// - v1.2: https://tools.ietf.org/html/rfc5246#appendix-A.1
// - v1.3: https://tools.ietf.org/html/rfc8446#appendix-B.1
//
// Note: this indicates that '0' is 'invalid'. Given that we only care about
// the first byte of a long-lived connection this is irrelevant, since you must
// always start out with a client hello handshake which is '22'.
func PeekForTLS(conn net.Conn) (net.Conn, bool, error) {
br := bufio.NewReader(conn)
// Grab enough to read the first byte. Then drain the buffer so future
// reads can be direct.
peeked, err := br.Peek(1)
if err != nil {
return nil, false, err
} else if len(peeked) == 0 {
return conn, false, nil
}
peeked, err = br.Peek(br.Buffered())
if err != nil {
return nil, false, err
}
isTLS := (peeked[0] > RPCMaxTypeValue)
return &peekedConn{
Peeked: peeked,
Conn: conn,
}, isTLS, nil
}

237
agent/pool/peek_test.go Normal file
View File

@ -0,0 +1,237 @@
package pool
import (
"crypto/ecdsa"
"crypto/elliptic"
"crypto/rand"
"crypto/tls"
"crypto/x509"
"errors"
"fmt"
"io/ioutil"
"net"
"testing"
"time"
"github.com/hashicorp/consul/tlsutil"
"github.com/stretchr/testify/require"
)
func TestPeekForTLS_not_TLS(t *testing.T) {
type testcase struct {
name string
connData []byte
}
var cases []testcase
for _, rpcType := range []RPCType{
RPCConsul,
RPCRaft,
RPCMultiplex,
RPCTLS,
RPCMultiplexV2,
RPCSnapshot,
RPCGossip,
RPCTLSInsecure,
} {
cases = append(cases, testcase{
name: fmt.Sprintf("tcp rpc type byte %d", rpcType),
connData: []byte{byte(rpcType), 'h', 'e', 'l', 'l', 'o'},
})
}
for _, tc := range cases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
dead := time.Now().Add(1 * time.Second)
serverConn, clientConn, err := deadlineNetPipe(dead)
require.NoError(t, err)
go func() {
_, _ = clientConn.Write(tc.connData)
_ = clientConn.Close()
}()
defer serverConn.Close()
wrapped, isTLS, err := PeekForTLS(serverConn)
require.NoError(t, err)
require.False(t, isTLS)
all, err := ioutil.ReadAll(wrapped)
require.NoError(t, err)
require.Equal(t, tc.connData, all)
})
}
}
func TestPeekForTLS_actual_TLS(t *testing.T) {
type testcase struct {
name string
connData []byte
}
var cases []testcase
for _, rpcType := range []RPCType{
RPCConsul,
RPCRaft,
RPCMultiplex,
RPCTLS,
RPCMultiplexV2,
RPCSnapshot,
RPCGossip,
RPCTLSInsecure,
} {
cases = append(cases, testcase{
name: fmt.Sprintf("tcp rpc type byte %d", rpcType),
connData: []byte{byte(rpcType), 'h', 'e', 'l', 'l', 'o'},
})
}
for _, tc := range cases {
tc := tc
t.Run(tc.name, func(t *testing.T) {
testPeekForTLS_withTLS(t, tc.connData)
})
}
}
func testPeekForTLS_withTLS(t *testing.T, connData []byte) {
t.Helper()
cert, caPEM, err := generateTestCert("server.dc1.consul")
require.NoError(t, err)
roots := x509.NewCertPool()
require.True(t, roots.AppendCertsFromPEM(caPEM))
dead := time.Now().Add(1 * time.Second)
serverConn, clientConn, err := deadlineNetPipe(dead)
require.NoError(t, err)
var (
clientErrCh = make(chan error, 1)
serverErrCh = make(chan error, 1)
serverGotPayload []byte
)
go func(conn net.Conn) { // Client
config := &tls.Config{
MinVersion: tls.VersionTLS12,
RootCAs: roots,
ServerName: "server.dc1.consul",
NextProtos: []string{"foo/bar"},
}
tlsConn := tls.Client(conn, config)
defer tlsConn.Close()
if err := tlsConn.Handshake(); err != nil {
clientErrCh <- err
return
}
_, err = tlsConn.Write(connData)
clientErrCh <- err
}(clientConn)
go func(conn net.Conn) { // Server
defer conn.Close()
wrapped, isTLS, err := PeekForTLS(conn)
if err != nil {
serverErrCh <- err
return
} else if !isTLS {
serverErrCh <- errors.New("expected to have peeked TLS but did not")
return
}
config := &tls.Config{
MinVersion: tls.VersionTLS12,
RootCAs: roots,
Certificates: []tls.Certificate{cert},
ServerName: "server.dc1.consul",
NextProtos: []string{"foo/bar"},
}
tlsConn := tls.Server(wrapped, config)
defer tlsConn.Close()
if err := tlsConn.Handshake(); err != nil {
serverErrCh <- err
return
}
all, err := ioutil.ReadAll(tlsConn)
if err != nil {
serverErrCh <- err
return
}
serverGotPayload = all
serverErrCh <- nil
}(serverConn)
require.NoError(t, <-clientErrCh)
require.NoError(t, <-serverErrCh)
require.Equal(t, connData, serverGotPayload)
}
func deadlineNetPipe(deadline time.Time) (net.Conn, net.Conn, error) {
server, client := net.Pipe()
if err := server.SetDeadline(deadline); err != nil {
server.Close()
client.Close()
return nil, nil, err
}
if err := client.SetDeadline(deadline); err != nil {
server.Close()
client.Close()
return nil, nil, err
}
return server, client, nil
}
func generateTestCert(serverName string) (cert tls.Certificate, caPEM []byte, err error) {
// generate CA
serial, err := tlsutil.GenerateSerialNumber()
if err != nil {
return tls.Certificate{}, nil, err
}
signer, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader)
if err != nil {
return tls.Certificate{}, nil, err
}
ca, err := tlsutil.GenerateCA(signer, serial, 365, nil)
if err != nil {
return tls.Certificate{}, nil, err
}
// generate leaf
serial, err = tlsutil.GenerateSerialNumber()
if err != nil {
return tls.Certificate{}, nil, err
}
certificate, privateKey, err := tlsutil.GenerateCert(
signer,
ca,
serial,
"Test Cert Name",
365,
[]string{serverName},
nil,
[]x509.ExtKeyUsage{x509.ExtKeyUsageServerAuth},
)
if err != nil {
return tls.Certificate{}, nil, err
}
cert, err = tls.X509KeyPair([]byte(certificate), []byte(privateKey))
if err != nil {
return tls.Certificate{}, nil, err
}
return cert, []byte(ca), nil
}

48
agent/pool/peeked_conn.go Normal file
View File

@ -0,0 +1,48 @@
// Copyright 2017 Google Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Originally from: https://github.com/google/tcpproxy/blob/master/tcpproxy.go
// at f5c09fbedceb69e4b238dec52cdf9f2fe9a815e2
package pool
import "net"
// peekedConn is an incoming connection that has had some bytes read from it
// to determine how to route the connection. The Read method stitches
// the peeked bytes and unread bytes back together.
type peekedConn struct {
// Peeked are the bytes that have been read from Conn for the
// purposes of route matching, but have not yet been consumed
// by Read calls. It set to nil by Read when fully consumed.
Peeked []byte
// Conn is the underlying connection.
// It can be type asserted against *net.TCPConn or other types
// as needed. It should not be read from directly unless
// Peeked is nil.
net.Conn
}
func (c *peekedConn) Read(p []byte) (n int, err error) {
if len(c.Peeked) > 0 {
n = copy(p, c.Peeked)
c.Peeked = c.Peeked[n:]
if len(c.Peeked) == 0 {
c.Peeked = nil
}
return n, nil
}
return c.Conn.Read(p)
}

View File

@ -42,6 +42,7 @@ type Conn struct {
refCount int32
shouldClose int32
nodeName string
addr net.Addr
session muxSession
lastUsed time.Time
@ -138,12 +139,24 @@ type ConnPool struct {
// TLSConfigurator
TLSConfigurator *tlsutil.Configurator
// GatewayResolver is a function that returns a suitable random mesh
// gateway address for dialing servers in a given DC. This is only
// needed if wan federation via mesh gateways is enabled.
GatewayResolver func(string) string
// Datacenter is the datacenter of the current agent.
Datacenter string
// ForceTLS is used to enforce outgoing TLS verification
ForceTLS bool
// Server should be set to true if this connection pool is configured in a
// server instead of a client.
Server bool
sync.Mutex
// pool maps an address to a open connection
// pool maps a nodeName+address to a open connection
pool map[string]*Conn
// limiter is used to throttle the number of connect attempts
@ -196,14 +209,20 @@ 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, useTLS bool) (*Conn, error) {
func (p *ConnPool) acquire(dc string, nodeName string, addr net.Addr, version int, useTLS bool) (*Conn, error) {
if nodeName == "" {
return nil, fmt.Errorf("pool: ConnPool.acquire requires a node name")
}
addrStr := addr.String()
poolKey := nodeName + ":" + addrStr
// Check to see if there's a pooled connection available. This is up
// here since it should the vastly more common case than the rest
// of the code here.
p.Lock()
c := p.pool[addrStr]
c := p.pool[poolKey]
if c != nil {
c.markForUse()
p.Unlock()
@ -225,7 +244,7 @@ func (p *ConnPool) acquire(dc string, addr net.Addr, version int, useTLS bool) (
// 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, useTLS)
c, err := p.getNewConn(dc, nodeName, addr, version, useTLS)
p.Lock()
delete(p.limiter, addrStr)
close(wait)
@ -234,7 +253,7 @@ func (p *ConnPool) acquire(dc string, addr net.Addr, version int, useTLS bool) (
return nil, err
}
p.pool[addrStr] = c
p.pool[poolKey] = c
p.Unlock()
return c, nil
}
@ -249,7 +268,7 @@ func (p *ConnPool) acquire(dc string, addr net.Addr, version int, useTLS bool) (
// See if the lead thread was able to get us a connection.
p.Lock()
if c := p.pool[addrStr]; c != nil {
if c := p.pool[poolKey]; c != nil {
c.markForUse()
p.Unlock()
return c, nil
@ -267,27 +286,91 @@ type HalfCloser interface {
// DialTimeout is used to establish a raw connection to the given server, with
// given connection timeout. It also writes RPCTLS as the first byte.
func (p *ConnPool) DialTimeout(dc string, addr net.Addr, timeout time.Duration, useTLS bool) (net.Conn, HalfCloser, error) {
func (p *ConnPool) DialTimeout(
dc string,
nodeName string,
addr net.Addr,
timeout time.Duration,
useTLS bool,
actualRPCType RPCType,
) (net.Conn, HalfCloser, error) {
p.once.Do(p.init)
return DialTimeoutWithRPCType(dc, addr, p.SrcAddr, timeout, useTLS || p.ForceTLS, p.TLSConfigurator.OutgoingRPCWrapper(), RPCTLS)
if p.Server && p.GatewayResolver != nil && p.TLSConfigurator != nil && dc != p.Datacenter {
// NOTE: TLS is required on this branch.
return DialTimeoutWithRPCTypeViaMeshGateway(
dc,
nodeName,
addr,
p.SrcAddr,
timeout,
p.TLSConfigurator.OutgoingALPNRPCWrapper(),
actualRPCType,
RPCTLS,
// gateway stuff
p.Server,
p.TLSConfigurator,
p.GatewayResolver,
p.Datacenter,
)
}
return DialTimeoutWithRPCTypeDirectly(
dc,
nodeName,
addr,
p.SrcAddr,
timeout,
useTLS || p.ForceTLS,
p.TLSConfigurator.OutgoingRPCWrapper(),
actualRPCType,
RPCTLS,
)
}
// DialTimeoutInsecure is used to establish a raw connection to the given
// server, with given connection timeout. It also writes RPCTLSInsecure as the
// first byte to indicate that the client cannot provide a certificate. This is
// so far only used for AutoEncrypt.Sign.
func (p *ConnPool) DialTimeoutInsecure(dc string, addr net.Addr, timeout time.Duration, wrapper tlsutil.DCWrapper) (net.Conn, HalfCloser, error) {
func (p *ConnPool) DialTimeoutInsecure(
dc string,
nodeName string,
addr net.Addr,
timeout time.Duration,
wrapper tlsutil.DCWrapper,
) (net.Conn, HalfCloser, error) {
p.once.Do(p.init)
if wrapper == nil {
return nil, nil, fmt.Errorf("wrapper cannot be nil")
} else if dc != p.Datacenter {
return nil, nil, fmt.Errorf("insecure dialing prohibited between datacenters")
}
return DialTimeoutWithRPCType(dc, addr, p.SrcAddr, timeout, true, wrapper, RPCTLSInsecure)
return DialTimeoutWithRPCTypeDirectly(
dc,
nodeName,
addr,
p.SrcAddr,
timeout,
true,
wrapper,
RPCTLSInsecure,
RPCTLSInsecure,
)
}
func DialTimeoutWithRPCType(dc string, addr net.Addr, src *net.TCPAddr, timeout time.Duration, useTLS bool, wrapper tlsutil.DCWrapper, rpcType RPCType) (net.Conn, HalfCloser, error) {
func DialTimeoutWithRPCTypeDirectly(
dc string,
nodeName string,
addr net.Addr,
src *net.TCPAddr,
timeout time.Duration,
useTLS bool,
wrapper tlsutil.DCWrapper,
actualRPCType RPCType,
tlsRPCType RPCType,
) (net.Conn, HalfCloser, error) {
// Try to dial the conn
d := &net.Dialer{LocalAddr: src, Timeout: timeout}
conn, err := d.Dial("tcp", addr.String())
@ -308,7 +391,7 @@ func DialTimeoutWithRPCType(dc string, addr net.Addr, src *net.TCPAddr, timeout
// Check if TLS is enabled
if (useTLS) && wrapper != nil {
// Switch the connection into TLS mode
if _, err := conn.Write([]byte{byte(rpcType)}); err != nil {
if _, err := conn.Write([]byte{byte(tlsRPCType)}); err != nil {
conn.Close()
return nil, nil, err
}
@ -327,27 +410,107 @@ func DialTimeoutWithRPCType(dc string, addr net.Addr, src *net.TCPAddr, timeout
}
}
// Send the type-byte for the protocol if one is required.
//
// When using insecure TLS there is no inner type-byte as these connections
// aren't wrapped like the standard TLS ones are.
if tlsRPCType != RPCTLSInsecure {
if _, err := conn.Write([]byte{byte(actualRPCType)}); err != nil {
conn.Close()
return nil, nil, err
}
}
return conn, hc, nil
}
// DialTimeoutWithRPCTypeViaMeshGateway dials the destination node and sets up
// the connection to be the correct RPC type using ALPN. This currently is
// exclusively used to dial other servers in foreign datacenters via mesh
// gateways.
//
// NOTE: There is a close mirror of this method in agent/consul/wanfed/wanfed.go:dial
func DialTimeoutWithRPCTypeViaMeshGateway(
dc string,
nodeName string,
addr net.Addr,
src *net.TCPAddr,
timeout time.Duration,
wrapper tlsutil.ALPNWrapper,
actualRPCType RPCType,
tlsRPCType RPCType,
// gateway stuff
dialingFromServer bool,
tlsConfigurator *tlsutil.Configurator,
gatewayResolver func(string) string,
thisDatacenter string,
) (net.Conn, HalfCloser, error) {
if !dialingFromServer {
return nil, nil, fmt.Errorf("must dial via mesh gateways from a server agent")
} else if gatewayResolver == nil {
return nil, nil, fmt.Errorf("gatewayResolver is nil")
} else if tlsConfigurator == nil {
return nil, nil, fmt.Errorf("tlsConfigurator is nil")
} else if dc == thisDatacenter {
return nil, nil, fmt.Errorf("cannot dial servers in the same datacenter via a mesh gateway")
} else if wrapper == nil {
return nil, nil, fmt.Errorf("cannot dial via a mesh gateway when outgoing TLS is disabled")
}
nextProto := actualRPCType.ALPNString()
if nextProto == "" {
return nil, nil, fmt.Errorf("rpc type %d cannot be routed through a mesh gateway", actualRPCType)
}
gwAddr := gatewayResolver(dc)
if gwAddr == "" {
return nil, nil, structs.ErrDCNotAvailable
}
dialer := &net.Dialer{LocalAddr: src, Timeout: timeout}
rawConn, err := dialer.Dial("tcp", gwAddr)
if err != nil {
return nil, nil, err
}
if tcp, ok := rawConn.(*net.TCPConn); ok {
_ = tcp.SetKeepAlive(true)
_ = tcp.SetNoDelay(true)
}
// NOTE: now we wrap the connection in a TLS client.
tlsConn, err := wrapper(dc, nodeName, nextProto, rawConn)
if err != nil {
return nil, nil, err
}
var conn net.Conn = tlsConn
var hc HalfCloser
if tlsConn, ok := conn.(*tls.Conn); ok {
// Expose *tls.Conn CloseWrite method on HalfCloser
hc = tlsConn
}
return conn, hc, nil
}
// getNewConn is used to return a new connection
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, useTLS)
if err != nil {
return nil, err
func (p *ConnPool) getNewConn(dc string, nodeName string, addr net.Addr, version int, useTLS bool) (*Conn, error) {
if nodeName == "" {
return nil, fmt.Errorf("pool: ConnPool.getNewConn requires a node name")
}
// Switch the multiplexing based on version
var session muxSession
if version < 2 {
conn.Close()
return nil, fmt.Errorf("cannot make client connection, unsupported protocol version %d", version)
}
// Write the Consul multiplex byte to set the mode
if _, err := conn.Write([]byte{byte(RPCMultiplexV2)}); err != nil {
conn.Close()
// Get a new, raw connection and write the Consul multiplex byte to set the mode
conn, _, err := p.DialTimeout(dc, nodeName, addr, defaultDialTimeout, useTLS, RPCMultiplexV2)
if err != nil {
return nil, err
}
@ -361,6 +524,7 @@ func (p *ConnPool) getNewConn(dc string, addr net.Addr, version int, useTLS bool
// Wrap the connection
c := &Conn{
refCount: 1,
nodeName: nodeName,
addr: addr,
session: session,
clients: list.New(),
@ -373,14 +537,19 @@ func (p *ConnPool) getNewConn(dc string, addr net.Addr, version int, useTLS bool
// clearConn is used to clear any cached connection, potentially in response to an error
func (p *ConnPool) clearConn(conn *Conn) {
if conn.nodeName == "" {
panic("pool: ConnPool.acquire requires a node name")
}
// Ensure returned streams are closed
atomic.StoreInt32(&conn.shouldClose, 1)
// Clear from the cache
addrStr := conn.addr.String()
poolKey := conn.nodeName + ":" + addrStr
p.Lock()
if c, ok := p.pool[addrStr]; ok && c == conn {
delete(p.pool, addrStr)
if c, ok := p.pool[poolKey]; ok && c == conn {
delete(p.pool, poolKey)
}
p.Unlock()
@ -399,11 +568,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, useTLS bool) (*Conn, *StreamClient, error) {
func (p *ConnPool) getClient(dc string, nodeName 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, useTLS)
conn, err := p.acquire(dc, nodeName, addr, version, useTLS)
if err != nil {
return nil, nil, fmt.Errorf("failed to get conn: %v", err)
}
@ -425,11 +594,24 @@ 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, useTLS bool, args interface{}, reply interface{}) error {
func (p *ConnPool) RPC(
dc string,
nodeName string,
addr net.Addr,
version int,
method string,
useTLS bool,
args interface{},
reply interface{},
) error {
if nodeName == "" {
return fmt.Errorf("pool: ConnPool.RPC requires a node name")
}
if method == "AutoEncrypt.Sign" {
return p.rpcInsecure(dc, addr, method, args, reply)
return p.rpcInsecure(dc, nodeName, addr, method, args, reply)
} else {
return p.rpc(dc, addr, version, method, useTLS, args, reply)
return p.rpc(dc, nodeName, addr, version, method, useTLS, args, reply)
}
}
@ -438,9 +620,9 @@ func (p *ConnPool) RPC(dc string, addr net.Addr, version int, method string, use
// transparent for the consumer. The pool cannot be used because
// AutoEncrypt.Sign is a one-off call and it doesn't make sense to pool that
// connection if it is not being reused.
func (p *ConnPool) rpcInsecure(dc string, addr net.Addr, method string, args interface{}, reply interface{}) error {
func (p *ConnPool) rpcInsecure(dc string, nodeName string, addr net.Addr, method string, args interface{}, reply interface{}) error {
var codec rpc.ClientCodec
conn, _, err := p.DialTimeoutInsecure(dc, addr, 1*time.Second, p.TLSConfigurator.OutgoingRPCWrapper())
conn, _, err := p.DialTimeoutInsecure(dc, nodeName, addr, 1*time.Second, p.TLSConfigurator.OutgoingRPCWrapper())
if err != nil {
return fmt.Errorf("rpcinsecure error establishing connection: %v", err)
}
@ -455,11 +637,11 @@ func (p *ConnPool) rpcInsecure(dc string, addr net.Addr, method string, args int
return nil
}
func (p *ConnPool) rpc(dc string, addr net.Addr, version int, method string, useTLS bool, args interface{}, reply interface{}) error {
func (p *ConnPool) rpc(dc string, nodeName string, addr net.Addr, version int, method string, useTLS bool, args interface{}, reply interface{}) error {
p.once.Do(p.init)
// Get a usable client
conn, sc, err := p.getClient(dc, addr, version, useTLS)
conn, sc, err := p.getClient(dc, nodeName, addr, version, useTLS)
if err != nil {
return fmt.Errorf("rpc error getting client: %v", err)
}
@ -489,9 +671,9 @@ func (p *ConnPool) rpc(dc string, addr net.Addr, version int, method string, use
// Ping sends a Status.Ping message to the specified server and
// returns true if healthy, false if an error occurred
func (p *ConnPool) Ping(dc string, addr net.Addr, version int, useTLS bool) (bool, error) {
func (p *ConnPool) Ping(dc string, nodeName string, addr net.Addr, version int, useTLS bool) (bool, error) {
var out struct{}
err := p.RPC(dc, addr, version, "Status.Ping", useTLS, struct{}{}, &out)
err := p.RPC(dc, nodeName, addr, version, "Status.Ping", useTLS, struct{}{}, &out)
return err == nil, err
}

View File

@ -7,6 +7,7 @@ import (
"github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/agent/local"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/go-hclog"
)
@ -65,7 +66,8 @@ type ManagerConfig struct {
// for now and cleaner than passing the entire RuntimeConfig.
Source *structs.QuerySource
// logger is the agent's logger to be used for logging logs.
Logger hclog.Logger
Logger hclog.Logger
TLSConfigurator *tlsutil.Configurator
}
// NewManager constructs a manager from the provided agent cache.
@ -184,6 +186,9 @@ func (m *Manager) ensureProxyServiceLocked(ns *structs.NodeService, token string
state.logger = m.Logger
state.cache = m.Cache
state.source = m.Source
if m.TLSConfigurator != nil {
state.serverSNIFn = m.TLSConfigurator.ServerSNI
}
ch, err := state.Watch()
if err != nil {

View File

@ -105,6 +105,7 @@ func TestManager_BasicLifecycle(t *testing.T) {
ID: "web-sidecar-proxy",
Service: "web-sidecar-proxy",
Port: 9999,
Meta: map[string]string{},
Proxy: structs.ConnectProxyConfig{
DestinationServiceID: "web",
DestinationServiceName: "web",
@ -197,6 +198,7 @@ func TestManager_BasicLifecycle(t *testing.T) {
Address: webProxy.Address,
Port: webProxy.Port,
Proxy: mustCopyProxyConfig(t, webProxy),
ServiceMeta: webProxy.Meta,
TaggedAddresses: make(map[string]structs.ServiceAddress),
Roots: roots,
ConnectProxy: configSnapshotConnectProxy{
@ -241,6 +243,7 @@ func TestManager_BasicLifecycle(t *testing.T) {
Address: webProxy.Address,
Port: webProxy.Port,
Proxy: mustCopyProxyConfig(t, webProxy),
ServiceMeta: webProxy.Meta,
TaggedAddresses: make(map[string]structs.ServiceAddress),
Roots: roots,
ConnectProxy: configSnapshotConnectProxy{
@ -328,7 +331,7 @@ func testManager_BasicLifecycle(
state.TriggerSyncChanges = func() {}
// Create manager
m, err := NewManager(ManagerConfig{c, state, source, logger})
m, err := NewManager(ManagerConfig{c, state, source, logger, nil})
require.NoError(err)
// And run it

View File

@ -34,25 +34,64 @@ func (c *configSnapshotConnectProxy) IsEmpty() bool {
}
type configSnapshotMeshGateway struct {
// map of service id to a cancel function. This cancel function is tied to the watch of
// connect enabled services for the given id. If the main datacenter services watch would
// indicate the removal of a service all together we then cancel watching that service for
// its connect endpoints.
// WatchedServices is a map of service id to a cancel function. This cancel
// function is tied to the watch of connect enabled services for the given
// id. If the main datacenter services watch would indicate the removal of
// a service all together we then cancel watching that service for its
// connect endpoints.
WatchedServices map[structs.ServiceID]context.CancelFunc
// Indicates that the watch on the datacenters services has completed. Even when there
// are no connect services, this being set (and the Connect roots being available) will be enough for
// the config snapshot to be considered valid. In the case of Envoy, this allows it to start its listeners
// even when no services would be proxied and allow its health check to pass.
// WatchedServicesSet indicates that the watch on the datacenters services
// has completed. Even when there are no connect services, this being set
// (and the Connect roots being available) will be enough for the config
// snapshot to be considered valid. In the case of Envoy, this allows it to
// start its listeners even when no services would be proxied and allow its
// health check to pass.
WatchedServicesSet bool
// map of datacenter name to a cancel function. This cancel function is tied
// to the watch of mesh-gateway services in that datacenter.
// WatchedDatacenters is a map of datacenter name to a cancel function.
// This cancel function is tied to the watch of mesh-gateway services in
// that datacenter.
WatchedDatacenters map[string]context.CancelFunc
// map of service id to the service instances of that service in the local datacenter
// ServiceGroups is a map of service id to the service instances of that
// service in the local datacenter.
ServiceGroups map[structs.ServiceID]structs.CheckServiceNodes
// map of service id to an associated service-resolver config entry for that service
// ServiceResolvers is a map of service id to an associated
// service-resolver config entry for that service.
ServiceResolvers map[structs.ServiceID]*structs.ServiceResolverConfigEntry
// map of datacenter names to services of kind mesh-gateway in that datacenter
// GatewayGroups is a map of datacenter names to services of kind
// mesh-gateway in that datacenter.
GatewayGroups map[string]structs.CheckServiceNodes
// FedStateGateways is a map of datacenter names to mesh gateways in that
// datacenter.
FedStateGateways map[string]structs.CheckServiceNodes
// ConsulServers is the list of consul servers in this datacenter.
ConsulServers structs.CheckServiceNodes
}
func (c *configSnapshotMeshGateway) Datacenters() []string {
sz1, sz2 := len(c.GatewayGroups), len(c.FedStateGateways)
sz := sz1
if sz2 > sz1 {
sz = sz2
}
dcs := make([]string, 0, sz)
for dc, _ := range c.GatewayGroups {
dcs = append(dcs, dc)
}
for dc, _ := range c.FedStateGateways {
if _, ok := c.GatewayGroups[dc]; !ok {
dcs = append(dcs, dc)
}
}
return dcs
}
func (c *configSnapshotMeshGateway) IsEmpty() bool {
@ -64,7 +103,9 @@ func (c *configSnapshotMeshGateway) IsEmpty() bool {
len(c.WatchedDatacenters) == 0 &&
len(c.ServiceGroups) == 0 &&
len(c.ServiceResolvers) == 0 &&
len(c.GatewayGroups) == 0
len(c.GatewayGroups) == 0 &&
len(c.FedStateGateways) == 0 &&
len(c.ConsulServers) == 0
}
// ConfigSnapshot captures all the resulting config needed for a proxy instance.
@ -76,11 +117,13 @@ type ConfigSnapshot struct {
ProxyID structs.ServiceID
Address string
Port int
ServiceMeta map[string]string
TaggedAddresses map[string]structs.ServiceAddress
Proxy structs.ConnectProxyConfig
Datacenter string
Roots *structs.IndexedCARoots
ServerSNIFn ServerSNIFunc
Roots *structs.IndexedCARoots
// connect-proxy specific
ConnectProxy configSnapshotConnectProxy
@ -97,6 +140,11 @@ func (s *ConfigSnapshot) Valid() bool {
case structs.ServiceKindConnectProxy:
return s.Roots != nil && s.ConnectProxy.Leaf != nil
case structs.ServiceKindMeshGateway:
if s.ServiceMeta[structs.MetaWANFederationKey] == "1" {
if len(s.MeshGateway.ConsulServers) == 0 {
return false
}
}
return s.Roots != nil && (s.MeshGateway.WatchedServicesSet || len(s.MeshGateway.ServiceGroups) > 0)
default:
return false

View File

@ -23,17 +23,19 @@ type CacheNotifier interface {
}
const (
coalesceTimeout = 200 * time.Millisecond
rootsWatchID = "roots"
leafWatchID = "leaf"
intentionsWatchID = "intentions"
serviceListWatchID = "service-list"
datacentersWatchID = "datacenters"
serviceResolversWatchID = "service-resolvers"
svcChecksWatchIDPrefix = cachetype.ServiceHTTPChecksName + ":"
serviceIDPrefix = string(structs.UpstreamDestTypeService) + ":"
preparedQueryIDPrefix = string(structs.UpstreamDestTypePreparedQuery) + ":"
defaultPreparedQueryPollInterval = 30 * time.Second
coalesceTimeout = 200 * time.Millisecond
rootsWatchID = "roots"
leafWatchID = "leaf"
intentionsWatchID = "intentions"
serviceListWatchID = "service-list"
federationStateListGatewaysWatchID = "federation-state-list-mesh-gateways"
consulServerListWatchID = "consul-server-list"
datacentersWatchID = "datacenters"
serviceResolversWatchID = "service-resolvers"
svcChecksWatchIDPrefix = cachetype.ServiceHTTPChecksName + ":"
serviceIDPrefix = string(structs.UpstreamDestTypeService) + ":"
preparedQueryIDPrefix = string(structs.UpstreamDestTypePreparedQuery) + ":"
defaultPreparedQueryPollInterval = 30 * time.Second
)
// state holds all the state needed to maintain the config for a registered
@ -41,9 +43,10 @@ const (
// is discarded and a new one created.
type state struct {
// logger, source and cache are required to be set before calling Watch.
logger hclog.Logger
source *structs.QuerySource
cache CacheNotifier
logger hclog.Logger
source *structs.QuerySource
cache CacheNotifier
serverSNIFn ServerSNIFunc
// ctx and cancel store the context created during initWatches call
ctx context.Context
@ -54,6 +57,7 @@ type state struct {
proxyID structs.ServiceID
address string
port int
meta map[string]string
taggedAddresses map[string]structs.ServiceAddress
proxyCfg structs.ConnectProxyConfig
token string
@ -63,6 +67,8 @@ type state struct {
reqCh chan chan *ConfigSnapshot
}
type ServerSNIFunc func(dc, nodeName string) string
func copyProxyConfig(ns *structs.NodeService) (structs.ConnectProxyConfig, error) {
if ns == nil {
return structs.ConnectProxyConfig{}, nil
@ -114,12 +120,18 @@ func newState(ns *structs.NodeService, token string) (*state, error) {
taggedAddresses[k] = v
}
meta := make(map[string]string)
for k, v := range ns.Meta {
meta[k] = v
}
return &state{
kind: ns.Kind,
service: ns.Service,
proxyID: ns.CompoundServiceID(),
address: ns.Address,
port: ns.Port,
meta: meta,
taggedAddresses: taggedAddresses,
proxyCfg: proxyCfg,
token: token,
@ -263,7 +275,6 @@ func (s *state) initWatchesConnectProxy() error {
for _, u := range s.proxyCfg.Upstreams {
dc := s.source.Datacenter
if u.Datacenter != "" {
// TODO(rb): if we ASK for a specific datacenter, do we still use the chain?
dc = u.Datacenter
}
@ -365,6 +376,29 @@ func (s *state) initWatchesMeshGateway() error {
return err
}
if s.meta[structs.MetaWANFederationKey] == "1" {
// Conveniently we can just use this service meta attribute in one
// place here to set the machinery in motion and leave the conditional
// behavior out of the rest of the package.
err = s.cache.Notify(s.ctx, cachetype.FederationStateListMeshGatewaysName, &structs.DCSpecificRequest{
Datacenter: s.source.Datacenter,
QueryOptions: structs.QueryOptions{Token: s.token},
Source: *s.source,
}, federationStateListGatewaysWatchID, s.ch)
if err != nil {
return err
}
err = s.cache.Notify(s.ctx, cachetype.HealthServicesName, &structs.ServiceSpecificRequest{
Datacenter: s.source.Datacenter,
QueryOptions: structs.QueryOptions{Token: s.token},
ServiceName: structs.ConsulServiceName,
}, consulServerListWatchID, s.ch)
if err != nil {
return err
}
}
// Eventually we will have to watch connect enable instances for each service as well as the
// destination services themselves but those notifications will be setup later. However we
// cannot setup those watches until we know what the services are. from the service list
@ -405,9 +439,11 @@ func (s *state) initialConfigSnapshot() ConfigSnapshot {
ProxyID: s.proxyID,
Address: s.address,
Port: s.port,
ServiceMeta: s.meta,
TaggedAddresses: s.taggedAddresses,
Proxy: s.proxyCfg,
Datacenter: s.source.Datacenter,
ServerSNIFn: s.serverSNIFn,
}
switch s.kind {
@ -418,8 +454,8 @@ func (s *state) initialConfigSnapshot() ConfigSnapshot {
snap.ConnectProxy.WatchedGateways = make(map[string]map[string]context.CancelFunc)
snap.ConnectProxy.WatchedGatewayEndpoints = make(map[string]map[string]structs.CheckServiceNodes)
snap.ConnectProxy.WatchedServiceChecks = make(map[structs.ServiceID][]structs.CheckType)
snap.ConnectProxy.PreparedQueryEndpoints = make(map[string]structs.CheckServiceNodes)
snap.ConnectProxy.PreparedQueryEndpoints = make(map[string]structs.CheckServiceNodes) // TODO(rb): deprecated
case structs.ServiceKindMeshGateway:
snap.MeshGateway.WatchedServices = make(map[structs.ServiceID]context.CancelFunc)
snap.MeshGateway.WatchedDatacenters = make(map[string]context.CancelFunc)
@ -759,6 +795,12 @@ func (s *state) handleUpdateMeshGateway(u cache.UpdateEvent, snap *ConfigSnapsho
return fmt.Errorf("invalid type for response: %T", u.Result)
}
snap.Roots = roots
case federationStateListGatewaysWatchID:
dcIndexedNodes, ok := u.Result.(*structs.DatacenterIndexedCheckServiceNodes)
if !ok {
return fmt.Errorf("invalid type for response: %T", u.Result)
}
snap.MeshGateway.FedStateGateways = dcIndexedNodes.DatacenterNodes
case serviceListWatchID:
services, ok := u.Result.(*structs.IndexedServiceList)
if !ok {
@ -866,6 +908,27 @@ func (s *state) handleUpdateMeshGateway(u cache.UpdateEvent, snap *ConfigSnapsho
}
}
snap.MeshGateway.ServiceResolvers = resolvers
case consulServerListWatchID:
resp, ok := u.Result.(*structs.IndexedCheckServiceNodes)
if !ok {
return fmt.Errorf("invalid type for response: %T", u.Result)
}
// Do some initial sanity checks to avoid doing something dumb.
for _, csn := range resp.Nodes {
if csn.Service.Service != structs.ConsulServiceName {
return fmt.Errorf("expected service name %q but got %q",
structs.ConsulServiceName, csn.Service.Service)
}
if csn.Node.Datacenter != snap.Datacenter {
return fmt.Errorf("expected datacenter %q but got %q",
snap.Datacenter, csn.Node.Datacenter)
}
}
snap.MeshGateway.ConsulServers = resp.Nodes
default:
switch {
case strings.HasPrefix(u.CorrelationID, "connect-service:"):

View File

@ -967,14 +967,18 @@ func testConfigSnapshotDiscoveryChain(t testing.T, variation string, additionalE
}
func TestConfigSnapshotMeshGateway(t testing.T) *ConfigSnapshot {
return testConfigSnapshotMeshGateway(t, true)
return testConfigSnapshotMeshGateway(t, true, false)
}
func TestConfigSnapshotMeshGatewayUsingFederationStates(t testing.T) *ConfigSnapshot {
return testConfigSnapshotMeshGateway(t, true, true)
}
func TestConfigSnapshotMeshGatewayNoServices(t testing.T) *ConfigSnapshot {
return testConfigSnapshotMeshGateway(t, false)
return testConfigSnapshotMeshGateway(t, false, false)
}
func testConfigSnapshotMeshGateway(t testing.T, populateServices bool) *ConfigSnapshot {
func testConfigSnapshotMeshGateway(t testing.T, populateServices bool, useFederationStates bool) *ConfigSnapshot {
roots, _ := TestCerts(t)
snap := &ConfigSnapshot{
Kind: structs.ServiceKindMeshGateway,
@ -1020,6 +1024,13 @@ func testConfigSnapshotMeshGateway(t testing.T, populateServices bool) *ConfigSn
"dc2": TestGatewayNodesDC2(t),
},
}
if useFederationStates {
snap.MeshGateway.FedStateGateways = map[string]structs.CheckServiceNodes{
"dc2": TestGatewayNodesDC2(t),
}
delete(snap.MeshGateway.GatewayGroups, "dc2")
}
}
return snap

View File

@ -13,6 +13,7 @@ import (
func (a *Agent) retryJoinLAN() {
r := &retryJoiner{
variant: retryJoinSerfVariant,
cluster: "LAN",
addrs: a.config.RetryJoinLAN,
maxAttempts: a.config.RetryJoinMaxAttemptsLAN,
@ -26,9 +27,51 @@ func (a *Agent) retryJoinLAN() {
}
func (a *Agent) retryJoinWAN() {
if !a.config.ServerMode {
a.logger.Warn("(WAN) couldn't join: Err: Must be a server to join WAN cluster")
return
}
isPrimary := a.config.PrimaryDatacenter == a.config.Datacenter
var joinAddrs []string
if a.config.ConnectMeshGatewayWANFederationEnabled {
// When wanfed is activated each datacenter 100% relies upon flood-join
// to replicate the LAN members in a dc into the WAN pool. We
// completely hijack whatever the user configured to correctly
// implement the star-join.
//
// Elsewhere we enforce that start-join-wan and retry-join-wan cannot
// be set if wanfed is enabled so we don't have to emit any warnings
// related to that here.
if isPrimary {
// Wanfed requires that secondaries join TO the primary and the
// primary doesn't explicitly join down to the secondaries, so as
// such in the primary a retry-join operation is a no-op.
return
}
// First get a handle on dialing the primary
a.refreshPrimaryGatewayFallbackAddresses()
// Then "retry join" a special address via the gateway which is
// load balanced to all servers in the primary datacenter
//
// Since this address is merely a placeholder we use an address from the
// TEST-NET-1 block as described in https://tools.ietf.org/html/rfc5735#section-3
const placeholderIPAddress = "192.0.2.2"
joinAddrs = []string{
fmt.Sprintf("*.%s/%s", a.config.PrimaryDatacenter, placeholderIPAddress),
}
} else {
joinAddrs = a.config.RetryJoinWAN
}
r := &retryJoiner{
variant: retryJoinSerfVariant,
cluster: "WAN",
addrs: a.config.RetryJoinWAN,
addrs: joinAddrs,
maxAttempts: a.config.RetryJoinMaxAttemptsWAN,
interval: a.config.RetryJoinIntervalWAN,
join: a.JoinWAN,
@ -39,6 +82,27 @@ func (a *Agent) retryJoinWAN() {
}
}
func (a *Agent) refreshPrimaryGatewayFallbackAddresses() {
r := &retryJoiner{
variant: retryJoinMeshGatewayVariant,
cluster: "primary",
addrs: a.config.PrimaryGateways,
maxAttempts: 0,
interval: a.config.PrimaryGatewaysInterval,
join: func(addrs []string) (int, error) {
if err := a.RefreshPrimaryGatewayFallbackAddresses(addrs); err != nil {
return 0, err
}
return len(addrs), nil
},
logger: a.logger,
stopCh: a.PrimaryMeshGatewayAddressesReadyCh(),
}
if err := r.retryJoin(); err != nil {
a.retryJoinCh <- err
}
}
func newDiscover() (*discover.Discover, error) {
providers := make(map[string]discover.Provider)
for k, v := range discover.Providers {
@ -52,7 +116,7 @@ func newDiscover() (*discover.Discover, error) {
)
}
func retryJoinAddrs(disco *discover.Discover, cluster string, retryJoin []string, logger hclog.Logger) []string {
func retryJoinAddrs(disco *discover.Discover, variant, cluster string, retryJoin []string, logger hclog.Logger) []string {
addrs := []string{}
if disco == nil {
return addrs
@ -73,10 +137,17 @@ func retryJoinAddrs(disco *discover.Discover, cluster string, retryJoin []string
} else {
addrs = append(addrs, servers...)
if logger != nil {
logger.Info("Discovered servers",
"cluster", cluster,
"servers", strings.Join(servers, " "),
)
if variant == retryJoinMeshGatewayVariant {
logger.Info("Discovered mesh gateways",
"cluster", cluster,
"mesh_gateways", strings.Join(servers, " "),
)
} else {
logger.Info("Discovered servers",
"cluster", cluster,
"servers", strings.Join(servers, " "),
)
}
}
}
@ -88,9 +159,18 @@ func retryJoinAddrs(disco *discover.Discover, cluster string, retryJoin []string
return addrs
}
const (
retryJoinSerfVariant = "serf"
retryJoinMeshGatewayVariant = "mesh-gateway"
)
// retryJoiner is used to handle retrying a join until it succeeds or all
// retries are exhausted.
type retryJoiner struct {
// variant is either "serf" or "mesh-gateway" and just adjusts the log messaging
// emitted
variant string
// cluster is the name of the serf cluster, e.g. "LAN" or "WAN".
cluster string
@ -108,8 +188,10 @@ type retryJoiner struct {
// serf cluster.
join func([]string) (int, error)
// logger is the agent logger. Log messages should contain the
// "agent: " prefix.
// stopCh is an optional stop channel to exit the retry loop early
stopCh <-chan struct{}
// logger is the agent logger.
logger hclog.Logger
}
@ -123,32 +205,64 @@ func (r *retryJoiner) retryJoin() error {
return err
}
r.logger.Info("Retry join is supported for the following discovery methods",
"discovery_methods", strings.Join(disco.Names(), " "),
)
r.logger.Info("Joining cluster...")
if r.variant == retryJoinMeshGatewayVariant {
r.logger.Info("Refreshing mesh gateways is supported for the following discovery methods",
"discovery_methods", strings.Join(disco.Names(), " "),
)
r.logger.Info("Refreshing mesh gateways...")
} else {
r.logger.Info("Retry join is supported for the following discovery methods",
"discovery_methods", strings.Join(disco.Names(), " "),
)
r.logger.Info("Joining cluster...")
}
attempt := 0
for {
addrs := retryJoinAddrs(disco, r.cluster, r.addrs, r.logger)
addrs := retryJoinAddrs(disco, r.variant, r.cluster, r.addrs, r.logger)
if len(addrs) > 0 {
n, err := r.join(addrs)
if err == nil {
r.logger.Info("Join cluster completed. Synced with initial agents", "num_agents", n)
if r.variant == retryJoinMeshGatewayVariant {
r.logger.Info("Refreshing mesh gateways completed")
} else {
r.logger.Info("Join cluster completed. Synced with initial agents", "num_agents", n)
}
return nil
}
} else if len(addrs) == 0 {
err = fmt.Errorf("No servers to join")
if r.variant == retryJoinMeshGatewayVariant {
err = fmt.Errorf("No mesh gateways found")
} else {
err = fmt.Errorf("No servers to join")
}
}
attempt++
if r.maxAttempts > 0 && attempt > r.maxAttempts {
return fmt.Errorf("agent: max join %s retry exhausted, exiting", r.cluster)
if r.variant == retryJoinMeshGatewayVariant {
return fmt.Errorf("agent: max refresh of %s mesh gateways retry exhausted, exiting", r.cluster)
} else {
return fmt.Errorf("agent: max join %s retry exhausted, exiting", r.cluster)
}
}
r.logger.Warn("Join cluster failed, will retry",
"retry_interval", r.interval,
"error", err,
)
time.Sleep(r.interval)
if r.variant == retryJoinMeshGatewayVariant {
r.logger.Warn("Refreshing mesh gateways failed, will retry",
"retry_interval", r.interval,
"error", err,
)
} else {
r.logger.Warn("Join cluster failed, will retry",
"retry_interval", r.interval,
"error", err,
)
}
select {
case <-time.After(r.interval):
case <-r.stopCh:
return nil
}
}
}

View File

@ -48,7 +48,7 @@ func TestAgentRetryJoinAddrs(t *testing.T) {
var buf bytes.Buffer
logger := testutil.LoggerWithOutput(t, &buf)
output := retryJoinAddrs(d, "LAN", test.input, logger)
output := retryJoinAddrs(d, retryJoinSerfVariant, "LAN", test.input, logger)
bufout := buf.String()
require.Equal(t, test.expected, output, bufout)
if i == 4 {
@ -57,6 +57,6 @@ func TestAgentRetryJoinAddrs(t *testing.T) {
})
}
t.Run("handles nil discover", func(t *testing.T) {
require.Equal(t, []string{}, retryJoinAddrs(nil, "LAN", []string{"a"}, nil))
require.Equal(t, []string{}, retryJoinAddrs(nil, retryJoinSerfVariant, "LAN", []string{"a"}, nil))
})
}

View File

@ -61,7 +61,7 @@ type ManagerSerfCluster interface {
// Pinger is an interface wrapping client.ConnPool to prevent a cyclic import
// dependency.
type Pinger interface {
Ping(dc string, addr net.Addr, version int, useTLS bool) (bool, error)
Ping(dc, nodeName string, addr net.Addr, version int, useTLS bool) (bool, error)
}
// serverList is a local copy of the struct used to maintain the list of
@ -340,7 +340,7 @@ func (m *Manager) RebalanceServers() {
// while Serf detects the node has failed.
srv := l.servers[0]
ok, err := m.connPoolPinger.Ping(srv.Datacenter, srv.Addr, srv.Version, srv.UseTLS)
ok, err := m.connPoolPinger.Ping(srv.Datacenter, srv.ShortName, srv.Addr, srv.Version, srv.UseTLS)
if ok {
foundHealthyServer = true
break

View File

@ -33,7 +33,7 @@ type fauxConnPool struct {
failPct float64
}
func (cp *fauxConnPool) Ping(string, net.Addr, int, bool) (bool, error) {
func (cp *fauxConnPool) Ping(string, string, net.Addr, int, bool) (bool, error) {
var success bool
successProb := rand.Float64()
if successProb > cp.failPct {
@ -179,7 +179,7 @@ func test_reconcileServerList(maxServers int) (bool, error) {
// failPct of the servers for the reconcile. This
// allows for the selected server to no longer be
// healthy for the reconcile below.
if ok, _ := m.connPoolPinger.Ping(node.Datacenter, node.Addr, node.Version, node.UseTLS); ok {
if ok, _ := m.connPoolPinger.Ping(node.Datacenter, node.ShortName, node.Addr, node.Version, node.UseTLS); ok {
// Will still be present
healthyServers = append(healthyServers, node)
} else {

View File

@ -32,7 +32,7 @@ type fauxConnPool struct {
failAddr net.Addr
}
func (cp *fauxConnPool) Ping(dc string, addr net.Addr, version int, useTLS bool) (bool, error) {
func (cp *fauxConnPool) Ping(dc string, nodeName string, addr net.Addr, version int, useTLS bool) (bool, error) {
var success bool
successProb := rand.Float64()

View File

@ -93,17 +93,19 @@ func FloodJoins(logger hclog.Logger, addrFn FloodAddrFn, portFn FloodPortFn,
}
}
globalServerName := fmt.Sprintf("%s.%s", server.Name, server.Datacenter)
// Do the join!
n, err := globalSerf.Join([]string{addr}, true)
n, err := globalSerf.Join([]string{globalServerName + "/" + addr}, true)
if err != nil {
logger.Debug("Failed to flood-join server at address",
"server", server.Name,
"server", globalServerName,
"address", addr,
"error", err,
)
} else if n > 0 {
logger.Debug("Successfully performed flood-join for server at address",
"server", server.Name,
"server", globalServerName,
"address", addr,
)
}

View File

@ -49,7 +49,7 @@ func (a *Agent) sidecarServiceFromNodeService(ns *structs.NodeService, token str
if sidecar.Meta != nil {
// Meta is non-nil validate it before we add the special key so we can
// enforce that user cannot add a consul- prefix one.
if err := structs.ValidateMetadata(sidecar.Meta, false); err != nil {
if err := structs.ValidateServiceMetadata(sidecar.Kind, sidecar.Meta, false); err != nil {
return nil, nil, "", err
}
}

View File

@ -0,0 +1,143 @@
package structs
import (
"sort"
"time"
)
// FederationStateOp is the operation for a request related to federation states.
type FederationStateOp string
const (
FederationStateUpsert FederationStateOp = "upsert"
FederationStateDelete FederationStateOp = "delete"
)
// FederationStateRequest is used to upsert and delete federation states.
type FederationStateRequest struct {
// Datacenter is the target for this request.
Datacenter string
// Op is the type of operation being requested.
Op FederationStateOp
// State is the federation state to upsert or in the case of a delete
// only the State.Datacenter field should be set.
State *FederationState
// WriteRequest is a common struct containing ACL tokens and other
// write-related common elements for requests.
WriteRequest
}
// RequestDatacenter returns the datacenter for a given request.
func (c *FederationStateRequest) RequestDatacenter() string {
return c.Datacenter
}
// FederationStates is a list of federation states.
type FederationStates []*FederationState
// Sort sorts federation states by their datacenter.
func (listings FederationStates) Sort() {
sort.Slice(listings, func(i, j int) bool {
return listings[i].Datacenter < listings[j].Datacenter
})
}
// FederationState defines some WAN federation related state that should be
// cross-shared between all datacenters joined on the WAN. One record exists
// per datacenter.
type FederationState struct {
// Datacenter is the name of the datacenter.
Datacenter string
// MeshGateways is a snapshot of the catalog state for all mesh gateways in
// this datacenter.
MeshGateways CheckServiceNodes `json:",omitempty"`
// UpdatedAt keeps track of when this record was modified.
UpdatedAt time.Time
// PrimaryModifyIndex is the ModifyIndex of the original data as it exists
// in the primary datacenter.
PrimaryModifyIndex uint64
// RaftIndex is local raft data.
RaftIndex
}
// IsSame is used to compare two federation states for the purposes of
// anti-entropy.
func (c *FederationState) IsSame(other *FederationState) bool {
if c.Datacenter != other.Datacenter {
return false
}
// We don't include the UpdatedAt field in this comparison because that is
// only updated when we re-persist.
if len(c.MeshGateways) != len(other.MeshGateways) {
return false
}
// NOTE: we don't bother to sort these since the order is going to be
// already defined by how the catalog returns results which should be
// stable enough.
for i := 0; i < len(c.MeshGateways); i++ {
a := c.MeshGateways[i]
b := other.MeshGateways[i]
if !a.Node.IsSame(b.Node) {
return false
}
if !a.Service.IsSame(b.Service) {
return false
}
if len(a.Checks) != len(b.Checks) {
return false
}
for j := 0; j < len(a.Checks); j++ {
ca := a.Checks[j]
cb := b.Checks[j]
if !ca.IsSame(cb) {
return false
}
}
}
return true
}
// FederationStateQuery is used to query federation states.
type FederationStateQuery struct {
// Datacenter is the target this request is intended for.
Datacenter string
// TargetDatacenter is the name of a datacenter to fetch the federation state for.
TargetDatacenter string
// Options for queries
QueryOptions
}
// RequestDatacenter returns the datacenter for a given request.
func (c *FederationStateQuery) RequestDatacenter() string {
return c.TargetDatacenter
}
// FederationStateResponse is the response to a FederationStateQuery request.
type FederationStateResponse struct {
State *FederationState
QueryMeta
}
// IndexedFederationStates represents the list of all federation states.
type IndexedFederationStates struct {
States FederationStates
QueryMeta
}

View File

@ -67,6 +67,7 @@ const (
ACLAuthMethodSetRequestType = 27
ACLAuthMethodDeleteRequestType = 28
ChunkingStateType = 29
FederationStateRequestType = 30
)
const (
@ -98,6 +99,10 @@ const (
// MetaSegmentKey is the node metadata key used to store the node's network segment
MetaSegmentKey = "consul-network-segment"
// MetaWANFederationKey is the mesh gateway metadata key that indicates a
// mesh gateway is usable for wan federation.
MetaWANFederationKey = "consul-wan-federation"
// MaxLockDelay provides a maximum LockDelay value for
// a session. Any value above this will not be respected.
MaxLockDelay = 60 * time.Second
@ -115,6 +120,8 @@ const (
WildcardSpecifier = "*"
)
var allowedConsulMetaKeysForMeshGateway = map[string]struct{}{MetaWANFederationKey: struct{}{}}
var (
NodeMaintCheckID = NewCheckID(NodeMaint, nil)
)
@ -641,14 +648,30 @@ func (n *Node) IsSame(other *Node) bool {
reflect.DeepEqual(n.Meta, other.Meta)
}
// ValidateNodeMetadata validates a set of key/value pairs from the agent
// config for use on a Node.
func ValidateNodeMetadata(meta map[string]string, allowConsulPrefix bool) error {
return validateMetadata(meta, allowConsulPrefix, nil)
}
// ValidateServiceMetadata validates a set of key/value pairs from the agent config for use on a Service.
// ValidateMeta validates a set of key/value pairs from the agent config
func ValidateMetadata(meta map[string]string, allowConsulPrefix bool) error {
func ValidateServiceMetadata(kind ServiceKind, meta map[string]string, allowConsulPrefix bool) error {
switch kind {
case ServiceKindMeshGateway:
return validateMetadata(meta, allowConsulPrefix, allowedConsulMetaKeysForMeshGateway)
default:
return validateMetadata(meta, allowConsulPrefix, nil)
}
}
func validateMetadata(meta map[string]string, allowConsulPrefix bool, allowedConsulKeys map[string]struct{}) error {
if len(meta) > metaMaxKeyPairs {
return fmt.Errorf("Node metadata cannot contain more than %d key/value pairs", metaMaxKeyPairs)
}
for key, value := range meta {
if err := validateMetaPair(key, value, allowConsulPrefix); err != nil {
if err := validateMetaPair(key, value, allowConsulPrefix, allowedConsulKeys); err != nil {
return fmt.Errorf("Couldn't load metadata pair ('%s', '%s'): %s", key, value, err)
}
}
@ -674,7 +697,7 @@ func ValidateWeights(weights *Weights) error {
}
// validateMetaPair checks that the given key/value pair is in a valid format
func validateMetaPair(key, value string, allowConsulPrefix bool) error {
func validateMetaPair(key, value string, allowConsulPrefix bool, allowedConsulKeys map[string]struct{}) error {
if key == "" {
return fmt.Errorf("Key cannot be blank")
}
@ -684,8 +707,10 @@ func validateMetaPair(key, value string, allowConsulPrefix bool) error {
if len(key) > metaKeyMaxLength {
return fmt.Errorf("Key is too long (limit: %d characters)", metaKeyMaxLength)
}
if strings.HasPrefix(key, metaKeyReservedPrefix) && !allowConsulPrefix {
return fmt.Errorf("Key prefix '%s' is reserved for internal use", metaKeyReservedPrefix)
if strings.HasPrefix(key, metaKeyReservedPrefix) {
if _, ok := allowedConsulKeys[key]; !allowConsulPrefix && !ok {
return fmt.Errorf("Key prefix '%s' is reserved for internal use", metaKeyReservedPrefix)
}
}
if len(value) > metaValueMaxLength {
return fmt.Errorf("Value is too long (limit: %d characters)", metaValueMaxLength)
@ -1518,6 +1543,13 @@ func (nodes CheckServiceNodes) Shuffle() {
}
}
// ShallowClone duplicates the slice and underlying array.
func (nodes CheckServiceNodes) ShallowClone() CheckServiceNodes {
dup := make(CheckServiceNodes, len(nodes))
copy(dup, nodes)
return dup
}
// Filter removes nodes that are failing health checks (and any non-passing
// check if that option is selected). Note that this returns the filtered
// results AND modifies the receiver for performance.
@ -1710,6 +1742,11 @@ type IndexedCheckServiceNodes struct {
QueryMeta
}
type DatacenterIndexedCheckServiceNodes struct {
DatacenterNodes map[string]CheckServiceNodes
QueryMeta
}
type IndexedNodeDump struct {
Dump NodeDump
QueryMeta

View File

@ -1164,45 +1164,102 @@ func TestStructs_DirEntry_Clone(t *testing.T) {
}
}
func TestStructs_ValidateMetadata(t *testing.T) {
// Load a valid set of key/value pairs
meta := map[string]string{
"key1": "value1",
"key2": "value2",
}
// Should succeed
if err := ValidateMetadata(meta, false); err != nil {
t.Fatalf("err: %s", err)
}
// Should get error
meta = map[string]string{
"": "value1",
}
if err := ValidateMetadata(meta, false); !strings.Contains(err.Error(), "Couldn't load metadata pair") {
t.Fatalf("should have failed")
}
// Should get error
meta = make(map[string]string)
func TestStructs_ValidateServiceAndNodeMetadata(t *testing.T) {
tooMuchMeta := make(map[string]string)
for i := 0; i < metaMaxKeyPairs+1; i++ {
meta[string(i)] = "value"
tooMuchMeta[string(i)] = "value"
}
if err := ValidateMetadata(meta, false); !strings.Contains(err.Error(), "cannot contain more than") {
t.Fatalf("should have failed")
type testcase struct {
Meta map[string]string
AllowConsulPrefix bool
NodeError string
ServiceError string
GatewayError string
}
cases := map[string]testcase{
"should succeed": {
map[string]string{
"key1": "value1",
"key2": "value2",
},
false,
"",
"",
"",
},
"invalid key": {
map[string]string{
"": "value1",
},
false,
"Couldn't load metadata pair",
"Couldn't load metadata pair",
"Couldn't load metadata pair",
},
"too many keys": {
tooMuchMeta,
false,
"cannot contain more than",
"cannot contain more than",
"cannot contain more than",
},
"reserved key prefix denied": {
map[string]string{
metaKeyReservedPrefix + "key": "value1",
},
false,
"reserved for internal use",
"reserved for internal use",
"reserved for internal use",
},
"reserved key prefix allowed": {
map[string]string{
metaKeyReservedPrefix + "key": "value1",
},
true,
"",
"",
"",
},
"reserved key prefix allowed via whitelist just for gateway - " + MetaWANFederationKey: {
map[string]string{
MetaWANFederationKey: "value1",
},
false,
"reserved for internal use",
"reserved for internal use",
"",
},
}
// Should not error
meta = map[string]string{
metaKeyReservedPrefix + "key": "value1",
}
// Should fail
if err := ValidateMetadata(meta, false); err == nil || !strings.Contains(err.Error(), "reserved for internal use") {
t.Fatalf("err: %s", err)
}
// Should succeed
if err := ValidateMetadata(meta, true); err != nil {
t.Fatalf("err: %s", err)
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
t.Run("ValidateNodeMetadata", func(t *testing.T) {
err := ValidateNodeMetadata(tc.Meta, tc.AllowConsulPrefix)
if tc.NodeError == "" {
require.NoError(t, err)
} else {
requireErrorContains(t, err, tc.NodeError)
}
})
t.Run("ValidateServiceMetadata - typical", func(t *testing.T) {
err := ValidateServiceMetadata(ServiceKindTypical, tc.Meta, tc.AllowConsulPrefix)
if tc.ServiceError == "" {
require.NoError(t, err)
} else {
requireErrorContains(t, err, tc.ServiceError)
}
})
t.Run("ValidateServiceMetadata - mesh-gateway", func(t *testing.T) {
err := ValidateServiceMetadata(ServiceKindMeshGateway, tc.Meta, tc.AllowConsulPrefix)
if tc.GatewayError == "" {
require.NoError(t, err)
} else {
requireErrorContains(t, err, tc.GatewayError)
}
})
})
}
}
@ -1214,27 +1271,32 @@ func TestStructs_validateMetaPair(t *testing.T) {
Value string
Error string
AllowConsulPrefix bool
AllowConsulKeys map[string]struct{}
}{
// valid pair
{"key", "value", "", false},
{"key", "value", "", false, nil},
// invalid, blank key
{"", "value", "cannot be blank", false},
{"", "value", "cannot be blank", false, nil},
// allowed special chars in key name
{"k_e-y", "value", "", false},
{"k_e-y", "value", "", false, nil},
// disallowed special chars in key name
{"(%key&)", "value", "invalid characters", false},
{"(%key&)", "value", "invalid characters", false, nil},
// key too long
{longKey, "value", "Key is too long", false},
{longKey, "value", "Key is too long", false, nil},
// reserved prefix
{metaKeyReservedPrefix + "key", "value", "reserved for internal use", false},
{metaKeyReservedPrefix + "key", "value", "reserved for internal use", false, nil},
// reserved prefix, allowed
{metaKeyReservedPrefix + "key", "value", "", true},
{metaKeyReservedPrefix + "key", "value", "", true, nil},
// reserved prefix, not allowed via whitelist
{metaKeyReservedPrefix + "bad", "value", "reserved for internal use", false, map[string]struct{}{metaKeyReservedPrefix + "good": struct{}{}}},
// reserved prefix, allowed via whitelist
{metaKeyReservedPrefix + "good", "value", "", true, map[string]struct{}{metaKeyReservedPrefix + "good": struct{}{}}},
// value too long
{"key", longValue, "Value is too long", false},
{"key", longValue, "Value is too long", false, nil},
}
for _, pair := range pairs {
err := validateMetaPair(pair.Key, pair.Value, pair.AllowConsulPrefix)
err := validateMetaPair(pair.Key, pair.Value, pair.AllowConsulPrefix, pair.AllowConsulKeys)
if pair.Error == "" && err != nil {
t.Fatalf("should have succeeded: %v, %v", pair, err)
} else if pair.Error != "" && !strings.Contains(err.Error(), pair.Error) {
@ -1963,3 +2025,13 @@ func TestSnapshotRequestResponse_MsgpackEncodeDecode(t *testing.T) {
})
}
func requireErrorContains(t *testing.T, err error, expectedErrorMessage string) {
t.Helper()
if err == nil {
t.Fatal("An error is expected but got nil.")
}
if !strings.Contains(err.Error(), expectedErrorMessage) {
t.Fatalf("unexpected error: %v", err)
}
}

View File

@ -378,7 +378,7 @@ func (a *TestAgent) consulConfig() *consul.Config {
// Instead of relying on one set of ports to be sufficient we retry
// starting the agent with different ports on port conflict.
func randomPortsSource(tls bool) (src config.Source, returnPortsFn func()) {
ports := freeport.MustTake(6)
ports := freeport.MustTake(7)
var http, https int
if tls {
@ -400,6 +400,7 @@ func randomPortsSource(tls bool) (src config.Source, returnPortsFn func()) {
serf_lan = ` + strconv.Itoa(ports[3]) + `
serf_wan = ` + strconv.Itoa(ports[4]) + `
server = ` + strconv.Itoa(ports[5]) + `
grpc = ` + strconv.Itoa(ports[6]) + `
}
`,
}, func() { freeport.Return(ports) }

View File

@ -113,11 +113,16 @@ func makeExposeClusterName(destinationPort int) string {
// for a mesh gateway. This will include 1 cluster per remote datacenter as well as
// 1 cluster for each service subset.
func (s *Server) clustersFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot, token string) ([]proto.Message, error) {
datacenters := cfgSnap.MeshGateway.Datacenters()
// 1 cluster per remote dc + 1 cluster per local service (this is a lower bound - all subset specific clusters will be appended)
clusters := make([]proto.Message, 0, len(cfgSnap.MeshGateway.GatewayGroups)+len(cfgSnap.MeshGateway.ServiceGroups))
clusters := make([]proto.Message, 0, len(datacenters)+len(cfgSnap.MeshGateway.ServiceGroups))
// generate the remote dc clusters
for dc, _ := range cfgSnap.MeshGateway.GatewayGroups {
for _, dc := range datacenters {
if dc == cfgSnap.Datacenter {
continue // skip local
}
clusterName := connect.DatacenterSNI(dc, cfgSnap.Roots.TrustDomain)
cluster, err := s.makeMeshGatewayCluster(clusterName, cfgSnap)
@ -127,6 +132,30 @@ func (s *Server) clustersFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapsho
clusters = append(clusters, cluster)
}
if cfgSnap.ServiceMeta[structs.MetaWANFederationKey] == "1" && cfgSnap.ServerSNIFn != nil {
// Add all of the remote wildcard datacenter mappings for servers.
for _, dc := range datacenters {
clusterName := cfgSnap.ServerSNIFn(dc, "")
cluster, err := s.makeMeshGatewayCluster(clusterName, cfgSnap)
if err != nil {
return nil, err
}
clusters = append(clusters, cluster)
}
// And for the current datacenter, send all flavors appropriately.
for _, srv := range cfgSnap.MeshGateway.ConsulServers {
clusterName := cfgSnap.ServerSNIFn(cfgSnap.Datacenter, srv.Node.Node)
cluster, err := s.makeMeshGatewayCluster(clusterName, cfgSnap)
if err != nil {
return nil, err
}
clusters = append(clusters, cluster)
}
}
// generate the per-service clusters
for svc, _ := range cfgSnap.MeshGateway.ServiceGroups {
clusterName := connect.ServiceSNI(svc.ID, "", svc.NamespaceOrDefault(), cfgSnap.Datacenter, cfgSnap.Roots.TrustDomain)

View File

@ -249,6 +249,11 @@ func TestClustersFromSnapshot(t *testing.T) {
create: proxycfg.TestConfigSnapshotMeshGateway,
setup: nil,
},
{
name: "mesh-gateway-using-federation-states",
create: proxycfg.TestConfigSnapshotMeshGatewayUsingFederationStates,
setup: nil,
},
{
name: "mesh-gateway-no-services",
create: proxycfg.TestConfigSnapshotMeshGatewayNoServices,

View File

@ -171,19 +171,88 @@ func (s *Server) filterSubsetEndpoints(subset *structs.ServiceResolverSubset, en
}
func (s *Server) endpointsFromSnapshotMeshGateway(cfgSnap *proxycfg.ConfigSnapshot, token string) ([]proto.Message, error) {
resources := make([]proto.Message, 0, len(cfgSnap.MeshGateway.GatewayGroups)+len(cfgSnap.MeshGateway.ServiceGroups))
datacenters := cfgSnap.MeshGateway.Datacenters()
resources := make([]proto.Message, 0, len(datacenters)+len(cfgSnap.MeshGateway.ServiceGroups))
// generate the endpoints for the gateways in the remote datacenters
for dc, endpoints := range cfgSnap.MeshGateway.GatewayGroups {
clusterName := connect.DatacenterSNI(dc, cfgSnap.Roots.TrustDomain)
la := makeLoadAssignment(
clusterName,
[]loadAssignmentEndpointGroup{
{Endpoints: endpoints},
},
cfgSnap.Datacenter,
)
resources = append(resources, la)
for _, dc := range datacenters {
if dc == cfgSnap.Datacenter {
continue // skip local
}
endpoints, ok := cfgSnap.MeshGateway.GatewayGroups[dc]
if !ok {
endpoints, ok = cfgSnap.MeshGateway.FedStateGateways[dc]
if !ok { // not possible
s.Logger.Error("skipping mesh gateway endpoints because no definition found", "datacenter", dc)
continue
}
}
{ // standard connect
clusterName := connect.DatacenterSNI(dc, cfgSnap.Roots.TrustDomain)
la := makeLoadAssignment(
clusterName,
[]loadAssignmentEndpointGroup{
{Endpoints: endpoints},
},
cfgSnap.Datacenter,
)
resources = append(resources, la)
}
if cfgSnap.ServiceMeta[structs.MetaWANFederationKey] == "1" && cfgSnap.ServerSNIFn != nil {
clusterName := cfgSnap.ServerSNIFn(dc, "")
la := makeLoadAssignment(
clusterName,
[]loadAssignmentEndpointGroup{
{Endpoints: endpoints},
},
cfgSnap.Datacenter,
)
resources = append(resources, la)
}
}
if cfgSnap.ServiceMeta[structs.MetaWANFederationKey] == "1" && cfgSnap.ServerSNIFn != nil {
// generate endpoints for our servers
var allServersLbEndpoints []envoyendpoint.LbEndpoint
for _, srv := range cfgSnap.MeshGateway.ConsulServers {
clusterName := cfgSnap.ServerSNIFn(cfgSnap.Datacenter, srv.Node.Node)
addr, port := srv.BestAddress(false /*wan*/)
lbEndpoint := envoyendpoint.LbEndpoint{
HostIdentifier: &envoyendpoint.LbEndpoint_Endpoint{
Endpoint: &envoyendpoint.Endpoint{
Address: makeAddressPtr(addr, port),
},
},
HealthStatus: envoycore.HealthStatus_UNKNOWN,
}
cla := &envoy.ClusterLoadAssignment{
ClusterName: clusterName,
Endpoints: []envoyendpoint.LocalityLbEndpoints{{
LbEndpoints: []envoyendpoint.LbEndpoint{lbEndpoint},
}},
}
allServersLbEndpoints = append(allServersLbEndpoints, lbEndpoint)
resources = append(resources, cla)
}
// And add one catch all so that remote datacenters can dial ANY server
// in this datacenter without knowing its name.
resources = append(resources, &envoy.ClusterLoadAssignment{
ClusterName: cfgSnap.ServerSNIFn(cfgSnap.Datacenter, ""),
Endpoints: []envoyendpoint.LocalityLbEndpoints{{
LbEndpoints: allServersLbEndpoints,
}},
})
}
// Generate the endpoints for each service and its subsets

View File

@ -237,6 +237,11 @@ func Test_endpointsFromSnapshot(t *testing.T) {
create: proxycfg.TestConfigSnapshotMeshGateway,
setup: nil,
},
{
name: "mesh-gateway-using-federation-states",
create: proxycfg.TestConfigSnapshotMeshGatewayUsingFederationStates,
setup: nil,
},
{
name: "mesh-gateway-no-services",
create: proxycfg.TestConfigSnapshotMeshGatewayNoServices,

View File

@ -517,7 +517,11 @@ func (s *Server) makeGatewayListener(name, addr string, port int, cfgSnap *proxy
// TODO (mesh-gateway) - Do we need to create clusters for all the old trust domains as well?
// We need 1 Filter Chain per datacenter
for dc := range cfgSnap.MeshGateway.GatewayGroups {
datacenters := cfgSnap.MeshGateway.Datacenters()
for _, dc := range datacenters {
if dc == cfgSnap.Datacenter {
continue // skip local
}
clusterName := connect.DatacenterSNI(dc, cfgSnap.Roots.TrustDomain)
filterName := fmt.Sprintf("%s_%s", name, dc)
dcTCPProxy, err := makeTCPProxyFilter(filterName, clusterName, "mesh_gateway_remote_")
@ -535,6 +539,49 @@ func (s *Server) makeGatewayListener(name, addr string, port int, cfgSnap *proxy
})
}
if cfgSnap.ServiceMeta[structs.MetaWANFederationKey] == "1" && cfgSnap.ServerSNIFn != nil {
for _, dc := range datacenters {
if dc == cfgSnap.Datacenter {
continue // skip local
}
clusterName := cfgSnap.ServerSNIFn(dc, "")
filterName := fmt.Sprintf("%s_%s", name, dc)
dcTCPProxy, err := makeTCPProxyFilter(filterName, clusterName, "mesh_gateway_remote_")
if err != nil {
return nil, err
}
l.FilterChains = append(l.FilterChains, envoylistener.FilterChain{
FilterChainMatch: &envoylistener.FilterChainMatch{
ServerNames: []string{fmt.Sprintf("*.%s", clusterName)},
},
Filters: []envoylistener.Filter{
dcTCPProxy,
},
})
}
// Wildcard all flavors to each server.
for _, srv := range cfgSnap.MeshGateway.ConsulServers {
clusterName := cfgSnap.ServerSNIFn(cfgSnap.Datacenter, srv.Node.Node)
filterName := fmt.Sprintf("%s_%s", name, cfgSnap.Datacenter)
dcTCPProxy, err := makeTCPProxyFilter(filterName, clusterName, "mesh_gateway_local_server_")
if err != nil {
return nil, err
}
l.FilterChains = append(l.FilterChains, envoylistener.FilterChain{
FilterChainMatch: &envoylistener.FilterChainMatch{
ServerNames: []string{fmt.Sprintf("%s", clusterName)},
},
Filters: []envoylistener.Filter{
dcTCPProxy,
},
})
}
}
// This needs to get tacked on at the end as it has no
// matching and will act as a catch all
l.FilterChains = append(l.FilterChains, sniClusterChain)

View File

@ -223,6 +223,10 @@ func TestListenersFromSnapshot(t *testing.T) {
name: "mesh-gateway",
create: proxycfg.TestConfigSnapshotMeshGateway,
},
{
name: "mesh-gateway-using-federation-states",
create: proxycfg.TestConfigSnapshotMeshGatewayUsingFederationStates,
},
{
name: "mesh-gateway-no-services",
create: proxycfg.TestConfigSnapshotMeshGatewayNoServices,

View File

@ -0,0 +1,55 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "bar.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
}
}
},
"connectTimeout": "5s",
"outlierDetection": {
}
},
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "dc2.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
}
}
},
"connectTimeout": "5s",
"outlierDetection": {
}
},
{
"@type": "type.googleapis.com/envoy.api.v2.Cluster",
"name": "foo.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"type": "EDS",
"edsClusterConfig": {
"edsConfig": {
"ads": {
}
}
},
"connectTimeout": "5s",
"outlierDetection": {
}
}
],
"typeUrl": "type.googleapis.com/envoy.api.v2.Cluster",
"nonce": "00000001"
}

View File

@ -0,0 +1,145 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment",
"clusterName": "bar.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.6",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.7",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.8",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
}
]
}
]
},
{
"@type": "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment",
"clusterName": "dc2.internal.11111111-2222-3333-4444-555555555555.consul",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "198.18.1.1",
"portValue": 443
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "198.18.1.2",
"portValue": 443
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
}
]
}
]
},
{
"@type": "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment",
"clusterName": "foo.default.dc1.internal.11111111-2222-3333-4444-555555555555.consul",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.3",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.4",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.5",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
},
{
"endpoint": {
"address": {
"socketAddress": {
"address": "172.16.1.9",
"portValue": 2222
}
}
},
"healthStatus": "HEALTHY",
"loadBalancingWeight": 1
}
]
}
]
}
],
"typeUrl": "type.googleapis.com/envoy.api.v2.ClusterLoadAssignment",
"nonce": "00000001"
}

View File

@ -0,0 +1,54 @@
{
"versionInfo": "00000001",
"resources": [
{
"@type": "type.googleapis.com/envoy.api.v2.Listener",
"name": "default:1.2.3.4:8443",
"address": {
"socketAddress": {
"address": "1.2.3.4",
"portValue": 8443
}
},
"filterChains": [
{
"filterChainMatch": {
"serverNames": [
"*.dc2.internal.11111111-2222-3333-4444-555555555555.consul"
]
},
"filters": [
{
"name": "envoy.tcp_proxy",
"config": {
"cluster": "dc2.internal.11111111-2222-3333-4444-555555555555.consul",
"stat_prefix": "mesh_gateway_remote_default_dc2_tcp"
}
}
]
},
{
"filters": [
{
"name": "envoy.filters.network.sni_cluster"
},
{
"name": "envoy.tcp_proxy",
"config": {
"cluster": "",
"stat_prefix": "mesh_gateway_local_default_tcp"
}
}
]
}
],
"listenerFilters": [
{
"name": "envoy.listener.tls_inspector"
}
]
}
],
"typeUrl": "type.googleapis.com/envoy.api.v2.Listener",
"nonce": "00000001"
}

View File

@ -10,7 +10,7 @@ require (
github.com/hashicorp/go-hclog v0.12.0
github.com/hashicorp/go-rootcerts v1.0.2
github.com/hashicorp/go-uuid v1.0.1
github.com/hashicorp/serf v0.8.2
github.com/hashicorp/serf v0.9.0
github.com/mitchellh/mapstructure v1.1.2
github.com/stretchr/testify v1.4.0
)

View File

@ -32,15 +32,14 @@ github.com/hashicorp/go-syslog v1.0.0/go.mod h1:qPfqrKkXGihmCqbJM2mZgkZGvKG1dFdv
github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE=
github.com/hashicorp/go-uuid v1.0.1/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro=
github.com/hashicorp/go.net v0.0.1/go.mod h1:hjKkEWcCURg++eb33jQU7oqQcI9XDCnUzHA0oac0k90=
github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo=
github.com/hashicorp/golang-lru v0.5.0/go.mod h1:/m3WP610KZHVQ1SGc6re/UDhFvYD7pJ4Ao+sR/qLZy8=
github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64=
github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
github.com/hashicorp/memberlist v0.1.3 h1:EmmoJme1matNzb+hMpDuR/0sbJSUisxyqBGG676r31M=
github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
github.com/hashicorp/serf v0.8.2 h1:YZ7UKsJv+hKjqGVUUbtE3HNj79Eln2oQ75tniF6iPt0=
github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
github.com/hashicorp/mdns v1.0.1/go.mod h1:4gW7WsVCke5TE7EPeYliwHlRUyBtfCwuFwuMg2DmyNY=
github.com/hashicorp/memberlist v0.2.0 h1:WeeNspppWi5s1OFefTviPQueC/Bq8dONfvNjPhiEQKE=
github.com/hashicorp/memberlist v0.2.0/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE=
github.com/hashicorp/serf v0.9.0 h1:+Zd/16AJ9lxk9RzfTDyv/TLhZ8UerqYS0/+JGCIDaa0=
github.com/hashicorp/serf v0.9.0/go.mod h1:YL0HO+FifKOW2u1ke99DGVu1zhcpZzNwrLIqBC7vbYU=
github.com/kr/pretty v0.2.0 h1:s5hAObm+yFO5uHYt5dYjxi2rXrsnmRpJx4OYvIWUaQs=
github.com/kr/pretty v0.2.0/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI=
github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ=
@ -58,13 +57,13 @@ github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHX
github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU=
github.com/miekg/dns v1.0.14 h1:9jZdLNd/P4+SfEJ0TNyxYpsK8N4GtfylBLqtbYN1sbA=
github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg=
github.com/miekg/dns v1.1.26 h1:gPxPSwALAeHJSjarOs00QjVdV9QoBvc1D2ujQUr5BzU=
github.com/miekg/dns v1.1.26/go.mod h1:bPDLeHnStXmXAq1m/Ch/hvfNHr14JKNPMBo3VZKjuso=
github.com/mitchellh/cli v1.0.0/go.mod h1:hNIlj7HEI86fIcpObd7a0FcrxTWetlwJDGcceTlRvqc=
github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y=
github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0=
github.com/mitchellh/go-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0=
github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE=
github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
@ -86,21 +85,36 @@ github.com/stretchr/testify v1.4.0 h1:2E4SXV/wtOkTonXsotYi4li6zVWxYlZuYNCXe9XRJy
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3 h1:KYQXGkl6vs02hK7pK4eIbw0NpNPedieTSTEiJ//bwGs=
golang.org/x/crypto v0.0.0-20181029021203-45a5f77698d3/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392 h1:ACG4HJsFiNMf47Y4PeRoebLNy/2lXT9EtprMuTFWt1M=
golang.org/x/crypto v0.0.0-20190923035154-9ee001bba392/go.mod h1:/lpIB1dKB+9EgE3H3cr1v9wB50oz8l4C4h62xy7jSTY=
golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc h1:a3CU5tJYVj92DY2LaA1kUkrsqD5/3mLDhx2NcNqyW+0=
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg=
golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/net v0.0.0-20190923162816-aa69164e4478 h1:l5EDrHhldLYb3ZRHDUhXF7Om7MvYXnkV9/iQNo1lX6g=
golang.org/x/net v0.0.0-20190923162816-aa69164e4478/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58 h1:8gQV6CLnAEikrhgkHFbMAEhagSSnXWGV915qUMm9mrU=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5 h1:x6r4Jo0KNzOOzYd8lbcRsqjuqEASK6ob3auvWYM4/8U=
golang.org/x/sys v0.0.0-20181026203630-95b1ffbd15a5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191008105621-543471e840be h1:QAcqgptGM8IQBC9K/RC4o+O9YmqEm0diQn9QmZw/0mU=
golang.org/x/sys v0.0.0-20191008105621-543471e840be/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200116001909-b77594299b42/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9 h1:1/DFK4b7JH8DmkqhUk48onnSfrPzImPoVxuomtbT2nk=
golang.org/x/sys v0.0.0-20200124204421-9fbb57f87de9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk=
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
golang.org/x/tools v0.0.0-20190907020128-2ca718005c18/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo=
golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15 h1:YR8cESwS4TdDjEe65xsg0ogRM/Nc3DYOhEAlW+xobZo=

View File

@ -64,6 +64,7 @@ type cmd struct {
wanAddress string
deregAfterCritical string
bindAddresses map[string]string
exposeServers bool
meshGatewaySvcName string
}
@ -130,6 +131,9 @@ func (c *cmd) init() {
c.flags.StringVar(&c.meshGatewaySvcName, "service", "mesh-gateway",
"Service name to use for the registration")
c.flags.BoolVar(&c.exposeServers, "expose-servers", false,
"Expose the servers for WAN federation via this mesh gateway")
c.flags.StringVar(&c.deregAfterCritical, "deregister-after-critical", "6h",
"The amount of time the gateway services health check can be failing before being deregistered")
@ -235,6 +239,17 @@ func (c *cmd) Run(args []string) int {
}
c.client = client
if c.exposeServers {
if !c.meshGateway {
c.UI.Error("'-expose-servers' can only be used for mesh gateways")
return 1
}
if !c.register {
c.UI.Error("'-expose-servers' requires '-register'")
return 1
}
}
if c.register {
if !c.meshGateway {
c.UI.Error("Auto-Registration can only be used for mesh gateways")
@ -307,11 +322,17 @@ func (c *cmd) Run(args []string) int {
return 1
}
var meta map[string]string
if c.exposeServers {
meta = map[string]string{structs.MetaWANFederationKey: "1"}
}
svc := api.AgentServiceRegistration{
Kind: api.ServiceKindMeshGateway,
Name: c.meshGatewaySvcName,
Address: lanAddr,
Port: lanPort,
Meta: meta,
TaggedAddresses: taggedAddrs,
Proxy: proxyConf,
Check: &api.AgentServiceCheck{

View File

@ -33,6 +33,7 @@ type cmd struct {
days int
domain string
help string
node string
dnsnames flags.AppendSliceValue
ipaddresses flags.AppendSliceValue
prefix string
@ -44,6 +45,7 @@ func (c *cmd) init() {
c.flags.StringVar(&c.key, "key", "#DOMAIN#-agent-ca-key.pem", "Provide path to the key. Defaults to #DOMAIN#-agent-ca-key.pem.")
c.flags.BoolVar(&c.server, "server", false, "Generate server certificate.")
c.flags.BoolVar(&c.client, "client", false, "Generate client certificate.")
c.flags.StringVar(&c.node, "node", "", "When generating a server cert and this is set an additional dns name is included of the form <node>.server.<datacenter>.<domain>.")
c.flags.BoolVar(&c.cli, "cli", false, "Generate cli certificate.")
c.flags.IntVar(&c.days, "days", 365, "Provide number of days the certificate is valid for from now on. Defaults to 1 year.")
c.flags.StringVar(&c.dc, "dc", "dc1", "Provide the datacenter. Matters only for -server certificates. Defaults to dc1.")
@ -79,6 +81,11 @@ func (c *cmd) Run(args []string) int {
return 1
}
if c.node != "" && !c.server {
c.UI.Error("-node requires -server")
return 1
}
var DNSNames []string
var IPAddresses []net.IP
var extKeyUsage []x509.ExtKeyUsage
@ -99,6 +106,10 @@ func (c *cmd) Run(args []string) int {
if c.server {
name = fmt.Sprintf("server.%s.%s", c.dc, c.domain)
if c.node != "" {
nodeName := fmt.Sprintf("%s.server.%s.%s", c.node, c.dc, c.domain)
DNSNames = append(DNSNames, nodeName)
}
DNSNames = append(DNSNames, name)
DNSNames = append(DNSNames, "localhost")

View File

@ -48,6 +48,11 @@ func TestTlsCertCreateCommand_InvalidArgs(t *testing.T) {
"Please provide either -server, -client, or -cli"},
"client+cli": {[]string{"-client", "-cli"},
"Please provide either -server, -client, or -cli"},
"client+node": {[]string{"-client", "-node", "foo"},
"-node requires -server"},
"cli+node": {[]string{"-cli", "-node", "foo"},
"-node requires -server"},
}
for name, tc := range cases {
@ -102,13 +107,14 @@ func TestTlsCertCreateCommand_fileCreate(t *testing.T) {
},
[]net.IP{{127, 0, 0, 1}},
},
{"server1",
{"server1-with-node",
"server",
[]string{"-server"},
[]string{"-server", "-node", "mysrv"},
"dc1-server-consul-1.pem",
"dc1-server-consul-1-key.pem",
"server.dc1.consul",
[]string{
"mysrv.server.dc1.consul",
"server.dc1.consul",
"localhost",
},

6
go.mod
View File

@ -27,6 +27,7 @@ require (
github.com/gogo/protobuf v1.2.1
github.com/golang/protobuf v1.3.1
github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf
github.com/google/tcpproxy v0.0.0-20180808230851-dfa16c61dad2
github.com/hashicorp/consul/api v1.4.0
github.com/hashicorp/consul/sdk v0.4.0
github.com/hashicorp/go-bexpr v0.1.2
@ -46,12 +47,11 @@ require (
github.com/hashicorp/golang-lru v0.5.1
github.com/hashicorp/hcl v1.0.0
github.com/hashicorp/hil v0.0.0-20160711231837-1e86c6b523c5
github.com/hashicorp/mdns v1.0.1 // indirect
github.com/hashicorp/memberlist v0.1.6
github.com/hashicorp/memberlist v0.2.0
github.com/hashicorp/net-rpc-msgpackrpc v0.0.0-20151116020338-a14192a58a69
github.com/hashicorp/raft v1.1.2
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea
github.com/hashicorp/serf v0.8.5
github.com/hashicorp/serf v0.9.0
github.com/hashicorp/vault/api v1.0.4
github.com/hashicorp/yamux v0.0.0-20181012175058-2f1d1f20f75d
github.com/imdario/mergo v0.3.6

16
go.sum
View File

@ -105,6 +105,8 @@ github.com/google/go-querystring v0.0.0-20170111101155-53e6ce116135 h1:zLTLjkaOF
github.com/google/go-querystring v0.0.0-20170111101155-53e6ce116135/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck=
github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf h1:+RRA9JqSOZFfKrOeqr2z77+8R2RKyh8PG66dcu1V0ck=
github.com/google/gofuzz v0.0.0-20170612174753-24818f796faf/go.mod h1:HP5RmnzzSNb993RKQDq4+1A4ia9nllfqcQFTQJedwGI=
github.com/google/tcpproxy v0.0.0-20180808230851-dfa16c61dad2 h1:AtvtonGEH/fZK0XPNNBdB6swgy7Iudfx88wzyIpwqJ8=
github.com/google/tcpproxy v0.0.0-20180808230851-dfa16c61dad2/go.mod h1:DavVbd41y+b7ukKDmlnPR4nGYmkWXR6vHUkjQNiHPBs=
github.com/googleapis/gnostic v0.2.0 h1:l6N3VoaVzTncYYW+9yOz2LJJammFZGBO13sqgEhpy9g=
github.com/googleapis/gnostic v0.2.0/go.mod h1:sJBsCZ4ayReDTBIg8b9dl28c5xFWyhBTVRp3pOg5EKY=
github.com/gophercloud/gophercloud v0.0.0-20180828235145-f29afc2cceca h1:wobTb8SE189AuxzEKClyYxiI4nUGWlpVtl13eLiFlOE=
@ -183,10 +185,8 @@ github.com/hashicorp/mdns v1.0.0 h1:WhIgCr5a7AaVH6jPUwjtRuuE7/RDufnUvzIr48smyxs=
github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ=
github.com/hashicorp/mdns v1.0.1 h1:XFSOubp8KWB+Jd2PDyaX5xUd5bhSP/+pTDZVDMzZJM8=
github.com/hashicorp/mdns v1.0.1/go.mod h1:4gW7WsVCke5TE7EPeYliwHlRUyBtfCwuFwuMg2DmyNY=
github.com/hashicorp/memberlist v0.1.3 h1:EmmoJme1matNzb+hMpDuR/0sbJSUisxyqBGG676r31M=
github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I=
github.com/hashicorp/memberlist v0.1.6 h1:ouPxvwKYaNZe+eTcHxYP0EblPduVLvIPycul+vv8his=
github.com/hashicorp/memberlist v0.1.6/go.mod h1:5VDNHjqFMgEcclnwmkCnC99IPwxBmIsxwY8qn+Nl0H4=
github.com/hashicorp/memberlist v0.2.0 h1:WeeNspppWi5s1OFefTviPQueC/Bq8dONfvNjPhiEQKE=
github.com/hashicorp/memberlist v0.2.0/go.mod h1:MS2lj3INKhZjWNqd3N0m3J+Jxf3DAOnAH9VT3Sh9MUE=
github.com/hashicorp/net-rpc-msgpackrpc v0.0.0-20151116020338-a14192a58a69 h1:lc3c72qGlIMDqQpQH82Y4vaglRMMFdJbziYWriR4UcE=
github.com/hashicorp/net-rpc-msgpackrpc v0.0.0-20151116020338-a14192a58a69/go.mod h1:/z+jUGRBlwVpUZfjute9jWaF6/HuhjuFQuL1YXzVD1Q=
github.com/hashicorp/raft v1.1.1 h1:HJr7UE1x/JrJSc9Oy6aDBHtNHUUBHjcQjTgvUVihoZs=
@ -195,10 +195,8 @@ github.com/hashicorp/raft v1.1.2 h1:oxEL5DDeurYxLd3UbcY/hccgSPhLLpiBZ1YxtWEq59c=
github.com/hashicorp/raft v1.1.2/go.mod h1:vPAJM8Asw6u8LxC3eJCUZmRP/E4QmUGE1R7g7k8sG/8=
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea h1:xykPFhrBAS2J0VBzVa5e80b5ZtYuNQtgXjN40qBZlD4=
github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pNv7Wc3ycL6F5oOWn+tPGo2gWD4a5X+yp/ntwdKLjRk=
github.com/hashicorp/serf v0.8.2 h1:YZ7UKsJv+hKjqGVUUbtE3HNj79Eln2oQ75tniF6iPt0=
github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc=
github.com/hashicorp/serf v0.8.5 h1:ZynDUIQiA8usmRgPdGPHFdPnb1wgGI9tK3mO9hcAJjc=
github.com/hashicorp/serf v0.8.5/go.mod h1:UpNcs7fFbpKIyZaUuSW6EPiH+eZC7OuyFD+wc1oal+k=
github.com/hashicorp/serf v0.9.0 h1:+Zd/16AJ9lxk9RzfTDyv/TLhZ8UerqYS0/+JGCIDaa0=
github.com/hashicorp/serf v0.9.0/go.mod h1:YL0HO+FifKOW2u1ke99DGVu1zhcpZzNwrLIqBC7vbYU=
github.com/hashicorp/vault/api v1.0.4 h1:j08Or/wryXT4AcHj1oCbMd7IijXcKzYUGw59LGu9onU=
github.com/hashicorp/vault/api v1.0.4/go.mod h1:gDcqh3WGcR1cpF5AJz/B1UFheUEneMoIospckxBxk6Q=
github.com/hashicorp/vault/sdk v0.1.13 h1:mOEPeOhT7jl0J4AMl1E705+BcmeRs1VmKNb9F0sMLy8=
@ -271,10 +269,8 @@ github.com/mitchellh/go-testing-interface v0.0.0-20171004221916-a61a99592b77/go.
github.com/mitchellh/go-testing-interface v1.0.0 h1:fzU/JVNcaqHQEcVFAKeR41fkiLdIPrefOvVG1VZ96U0=
github.com/mitchellh/go-testing-interface v1.0.0/go.mod h1:kRemZodwjscx+RGhAo8eIhFbs2+BFgRtFPeD/KE+zxI=
github.com/mitchellh/go-wordwrap v1.0.0/go.mod h1:ZXFpozHsX6DPmq2I0TCekCxypsnAUbP2oI0UX1GXzOo=
github.com/mitchellh/gox v0.4.0/go.mod h1:Sd9lOJ0+aimLBi73mGofS1ycjY8lL3uZM3JPS42BGNg=
github.com/mitchellh/hashstructure v0.0.0-20170609045927-2bca23e0e452 h1:hOY53G+kBFhbYFpRVxHl5eS7laP6B1+Cq+Z9Dry1iMU=
github.com/mitchellh/hashstructure v0.0.0-20170609045927-2bca23e0e452/go.mod h1:QjSHrPWS+BGUVBYkbTZWEnOh3G1DutKwClXU/ABz6AQ=
github.com/mitchellh/iochan v1.0.0/go.mod h1:JwYml1nuB7xOzsp52dPpHFffvOCDupsG0QubkSMEySY=
github.com/mitchellh/mapstructure v0.0.0-20160808181253-ca63d7c062ee/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=
github.com/mitchellh/mapstructure v1.1.2 h1:fmNYVwqnSfB9mZU6OS2O6GsXM+wcskZDuKQzvN1EDeE=
github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y=

17
lib/slice.go Normal file
View File

@ -0,0 +1,17 @@
package lib
// StringSliceEqual compares two string slices for equality. Both the existence
// of the elements and the order of those elements matter for equality. Empty
// slices are treated identically to nil slices.
func StringSliceEqual(a, b []string) bool {
if len(a) != len(b) {
return false
}
for i := 0; i < len(a); i++ {
if a[i] != b[i] {
return false
}
}
return true
}

30
lib/slice_test.go Normal file
View File

@ -0,0 +1,30 @@
package lib
import (
"fmt"
"testing"
"github.com/stretchr/testify/require"
)
func TestStringSliceEqual(t *testing.T) {
for _, tc := range []struct {
a, b []string
equal bool
}{
{nil, nil, true},
{nil, []string{}, true},
{[]string{}, []string{}, true},
{[]string{"a"}, []string{"a"}, true},
{[]string{}, []string{"a"}, false},
{[]string{"a"}, []string{"a", "b"}, false},
{[]string{"a", "b"}, []string{"a", "b"}, true},
{[]string{"a", "b"}, []string{"b", "a"}, false},
} {
name := fmt.Sprintf("%#v =?= %#v", tc.a, tc.b)
t.Run(name, func(t *testing.T) {
require.Equal(t, tc.equal, StringSliceEqual(tc.a, tc.b))
require.Equal(t, tc.equal, StringSliceEqual(tc.b, tc.a))
})
}
}

View File

@ -1,50 +1,52 @@
package logging
const (
ACL string = "acl"
Agent string = "agent"
AntiEntropy string = "anti_entropy"
AutoEncrypt string = "auto_encrypt"
Autopilot string = "autopilot"
AWS string = "aws"
Azure string = "azure"
CA string = "ca"
CentralConfig string = "central_config"
ConfigEntry string = "config_entry"
Connect string = "connect"
Consul string = "consul"
ConsulClient string = "client"
ConsulServer string = "server"
Coordinate string = "coordinate"
DNS string = "dns"
Envoy string = "envoy"
FSM string = "fsm"
HTTP string = "http"
Intentions string = "intentions"
Internal string = "internal"
KV string = "kvs"
LAN string = "lan"
Leader string = "leader"
Legacy string = "legacy"
License string = "license"
Manager string = "manager"
Memberlist string = "memberlist"
MeshGateway string = "mesh_gateway"
Namespace string = "namespace"
Operator string = "operator"
PreparedQuery string = "prepared_query"
Proxy string = "proxy"
ProxyConfig string = "proxycfg"
Raft string = "raft"
Replication string = "replication"
Router string = "router"
RPC string = "rpc"
Serf string = "serf"
Session string = "session"
Sentinel string = "sentinel"
Snapshot string = "snapshot"
TLSUtil string = "tlsutil"
Transaction string = "txn"
WAN string = "wan"
Watch string = "watch"
ACL string = "acl"
Agent string = "agent"
AntiEntropy string = "anti_entropy"
AutoEncrypt string = "auto_encrypt"
Autopilot string = "autopilot"
AWS string = "aws"
Azure string = "azure"
CA string = "ca"
CentralConfig string = "central_config"
ConfigEntry string = "config_entry"
Connect string = "connect"
Consul string = "consul"
ConsulClient string = "client"
ConsulServer string = "server"
Coordinate string = "coordinate"
DNS string = "dns"
Envoy string = "envoy"
FederationState string = "federation_state"
FSM string = "fsm"
GatewayLocator string = "gateway_locator"
HTTP string = "http"
Intentions string = "intentions"
Internal string = "internal"
KV string = "kvs"
LAN string = "lan"
Leader string = "leader"
Legacy string = "legacy"
License string = "license"
Manager string = "manager"
Memberlist string = "memberlist"
MeshGateway string = "mesh_gateway"
Namespace string = "namespace"
Operator string = "operator"
PreparedQuery string = "prepared_query"
Proxy string = "proxy"
ProxyConfig string = "proxycfg"
Raft string = "raft"
Replication string = "replication"
Router string = "router"
RPC string = "rpc"
Serf string = "serf"
Session string = "session"
Sentinel string = "sentinel"
Snapshot string = "snapshot"
TLSUtil string = "tlsutil"
Transaction string = "txn"
WAN string = "wan"
Watch string = "watch"
)

22
test/hostname/Betty.cfg Normal file
View File

@ -0,0 +1,22 @@
[req]
prompt = no
distinguished_name = dn
req_extensions = v3_req
[dn]
C = US
ST = California
L = Los Angeles
O = End Point
OU = Testing
emailAddress = do-not-reply@hashicorp.com
CN = Betty
[v3_req]
basicConstraints = CA:FALSE
keyUsage = nonRepudiation, digitalSignature, keyEncipherment
subjectAltName = @alt_names
[alt_names]
DNS.1 = server.dc2.consul
DNS.2 = betty.server.dc2.consul

23
test/hostname/Betty.crt Normal file
View File

@ -0,0 +1,23 @@
-----BEGIN CERTIFICATE-----
MIID7DCCAtSgAwIBAgIBHDANBgkqhkiG9w0BAQUFADCBmTELMAkGA1UEBhMCVVMx
EzARBgNVBAgTCkNhbGlmb3JuaWExFDASBgNVBAcTC0xvcyBBbmdlbGVzMRkwFwYD
VQQKExBIYWhpQ29ycCBUZXN0IENBMQ0wCwYDVQQLEwRUZXN0MREwDwYDVQQDEwhD
ZXJ0QXV0aDEiMCAGCSqGSIb3DQEJARYTamFtZXNAaGFzaGljb3JwLmNvbTAgFw0x
OTEyMTEyMTQzMzlaGA8yMTE5MTExNzIxNDMzOVowgYMxDjAMBgNVBAMMBUJldHR5
MRMwEQYDVQQIDApDYWxpZm9ybmlhMQswCQYDVQQGEwJVUzEpMCcGCSqGSIb3DQEJ
ARYaZG8tbm90LXJlcGx5QGhhc2hpY29ycC5jb20xEjAQBgNVBAoMCUVuZCBQb2lu
dDEQMA4GA1UECwwHVGVzdGluZzCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoC
ggEBAPU04u9/94fgQZMwCHR9gX6yBfJV6m7hTxR7rQv8GXaaCYVVisH2NmW6KcrZ
hjUqsvPpm63vEZasYC2blqlLnQCmJyOemnx9v0WEX9SLM3w8ihjbGhSq6VqaCeGH
s3jaxe9Bx8anR1tWiz2AoEEP1SzHgBQv08swDdWZsFKqnXntwqKqZcegIQMelxW+
iofAtSRZcwhbQUrpgaarxStuvpxqt1y/rbS27H1cf9U4CLysKClOIIJE3l7rqKCb
R5uYyQd07nZC+R7/83TX1AGFvk55QujB9Pm9p6RbjHJWZ5CLPtpiQhpMwYw1JluN
1KSwnpDDreCWMw+yEchlAnpw3/cCAwEAAaNRME8wCQYDVR0TBAIwADALBgNVHQ8E
BAMCBeAwNQYDVR0RBC4wLIIRc2VydmVyLmRjMi5jb25zdWyCF2JldHR5LnNlcnZl
ci5kYzIuY29uc3VsMA0GCSqGSIb3DQEBBQUAA4IBAQBvGhMpUHmw3j7+sj0D+mCz
+bBhZH6HEpy6TLjS1GfO0/fyO2DIcPMHNTdNqmoDTt33scS53155jEhCI8Wtb6LY
Mvoo0wwnQtGvuqyscnJldAQ++08N2bjJq9iQoG1gB9oPWOxRe4tjbSoJNl1X3a0u
jwjKwOl0HX23WMy3S5mIKuOBuT79/nY/rVlFP1fsna4TKO1ocXjK5JnQ9TKdGTRH
9STT/RPIIQvWg+zeDS+ZlMocZEq7NT63d2BzH2ZiV6VRZM0PSyEixE0fqfxPxA2D
+fqeDl8iRR4tPIifkDFZLoMiHDa7Ciqh1hgdMUk1tkPZpxy+XP+AzI/K/3Tnceer
-----END CERTIFICATE-----

7
test/hostname/Betty.ext Normal file
View File

@ -0,0 +1,7 @@
basicConstraints = CA:FALSE
keyUsage = nonRepudiation, digitalSignature, keyEncipherment
subjectAltName = @alt_names
[alt_names]
DNS.1 = server.dc2.consul
DNS.2 = betty.server.dc2.consul

28
test/hostname/Betty.key Normal file
View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvwIBADANBgkqhkiG9w0BAQEFAASCBKkwggSlAgEAAoIBAQD1NOLvf/eH4EGT
MAh0fYF+sgXyVepu4U8Ue60L/Bl2mgmFVYrB9jZluinK2YY1KrLz6Zut7xGWrGAt
m5apS50Apicjnpp8fb9FhF/UizN8PIoY2xoUqulamgnhh7N42sXvQcfGp0dbVos9
gKBBD9Usx4AUL9PLMA3VmbBSqp157cKiqmXHoCEDHpcVvoqHwLUkWXMIW0FK6YGm
q8Urbr6cardcv620tux9XH/VOAi8rCgpTiCCRN5e66igm0ebmMkHdO52Qvke//N0
19QBhb5OeULowfT5vaekW4xyVmeQiz7aYkIaTMGMNSZbjdSksJ6Qw63gljMPshHI
ZQJ6cN/3AgMBAAECggEARpwMHVuENCRnvbK/PIxHlew9uiLzdyp6UzOqCSF3U6fX
xgV0B5aW44RQNJGfDABXt9U1d0i4j+Ljbz62i9myRFWUP7WUVvT+902/Kr1K/iOQ
wMeXIwx0Vhq1bbReAhc9mEAg/xt8eNjbD8LSYunkQRjR0P5UxtX3peKz25o17r3w
U5lpvbYzm/k376Dhr2RBr30jrrf2rh06+FQCc2dF2mK1j7+YKbIHK+BKQYtQeVyg
XYpJfJTsuHFojwZNGXEuidkGApuokTS0HiAuAjrCQsn4cUftXnUtE2HJgsCum/Bp
Kb74ahBbZCITXCRSKZCi6p9oFcHQ30JDCCz4Qy9HgQKBgQD/dzWYKzI29ihQmeLN
ntHRl4RTjO4LfCs6lr8ul5nFOcgGwSwaFaTbqq0oJefCqEH+wmH1Jbd5nfRi7PWr
uGibeZnLdiseHHMsvN8l6PY3tVCm3kJL5Ze2TY+n8/7eUPcmH60CFikqO53ahjV3
9PtUBr5BUe1xUJ6T4zegqZKWbwKBgQD1uC8PfrIMGLmq9l3x3T2pAbmfz0N3DfUs
ncY+JCQRkBkWJk3oW8ITBZagCwvg4AnhbGvNgbAaPGEQ9HL7f19ieJeHxEaVtTY2
kKDwelPHT06oCu2AZ8h1Zqfn55O/HtGO/MuTdFa9IKjGYJTUvSBy3VVd8gnK9MBV
fhUtEqNS+QKBgQC33NR18KDzbbcWS6sw0l2wu5xBhezN11BFmrl+jx3dFPkh42Ya
X/mHIBAAFUf4kaDt+nkGN18V6Nk7WdB3BwJC5AIMrb/arB8407bHUiPjdFvXvZ95
gITwcGI0PyfwWdWHWsTp+4klHENAQ9e3vlok37WOzahXJe78AUzIFUOrgQKBgQCb
qC3Htw67Mv6LGr6wdOKWqY0Ze4bVaHYj6V6oBuUCazI5IdLAmz/6JNQiVl0T+1jH
AJPZ/4m7VPx4bSJZx3p5OsNjMic0tzK8pioNrLBd1hORyDpj2VrXZEyBT+X8cF14
IxQjONOpw4KnCI+/pH9lxGhLtwQVGa6tec2YW/IyoQKBgQCMr00Z1/+edBh/s+Ho
p87Wwf3vRtRZLniVdc1jVk9raK6azrFS+vBzpkWZatLu5Grtwl/9HYNTu+AnfKGP
jyRkCx0i5qgEQobYkiAJeFocyDVbzaDdZBhTAINN9uaSDH1JpGNlIBxIflzT0adf
OCBbgQ6SaTH+MWvYJ1KJPsQVkw==
-----END PRIVATE KEY-----

22
test/hostname/Bob.cfg Normal file
View File

@ -0,0 +1,22 @@
[req]
prompt = no
distinguished_name = dn
req_extensions = v3_req
[dn]
C = US
ST = California
L = Los Angeles
O = End Point
OU = Testing
emailAddress = do-not-reply@hashicorp.com
CN = Bob
[v3_req]
basicConstraints = CA:FALSE
keyUsage = nonRepudiation, digitalSignature, keyEncipherment
subjectAltName = @alt_names
[alt_names]
DNS.1 = server.dc1.consul
DNS.2 = bob.server.dc1.consul

23
test/hostname/Bob.crt Normal file
View File

@ -0,0 +1,23 @@
-----BEGIN CERTIFICATE-----
MIID6DCCAtCgAwIBAgIBGzANBgkqhkiG9w0BAQUFADCBmTELMAkGA1UEBhMCVVMx
EzARBgNVBAgTCkNhbGlmb3JuaWExFDASBgNVBAcTC0xvcyBBbmdlbGVzMRkwFwYD
VQQKExBIYWhpQ29ycCBUZXN0IENBMQ0wCwYDVQQLEwRUZXN0MREwDwYDVQQDEwhD
ZXJ0QXV0aDEiMCAGCSqGSIb3DQEJARYTamFtZXNAaGFzaGljb3JwLmNvbTAgFw0x
OTEyMDQyMDMzMjhaGA8yMTE5MTExMDIwMzMyOFowgYExDDAKBgNVBAMMA0JvYjET
MBEGA1UECAwKQ2FsaWZvcm5pYTELMAkGA1UEBhMCVVMxKTAnBgkqhkiG9w0BCQEW
GmRvLW5vdC1yZXBseUBoYXNoaWNvcnAuY29tMRIwEAYDVQQKDAlFbmQgUG9pbnQx
EDAOBgNVBAsMB1Rlc3RpbmcwggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIB
AQC+TMR+iyWgqvEmaqDTS7AaK5M18oPF47dDPm/o6/RbPRDO1KfcXXaJCk14tTd2
BbgUPHyuOf5CfEQIBc3JgI8Aj4nTY56Fo7Zz0igEOd2tXBe0scx0dXZPrRnnUfg1
tG8kBJGYL4wR7Bd8N0xCpZK4+6NWyEkGmiTCI+NoVevhadGDrTlLbs+1GvzuufUB
OnVsam28beDfFI7JoGFpV/wbu93C3BUs2yg7wvHrAw2uvA0K5A05Vk+w61gW9bKW
HNGvOzTIr5ZWYFLYO2xNq/9vbmnX/teYiMWd7OkZbwTssbV2L9NJ0hML7fd48Rb9
3jjXAXCqHQgliqUZ45aTQEqlAgMBAAGjTzBNMAkGA1UdEwQCMAAwCwYDVR0PBAQD
AgXgMDMGA1UdEQQsMCqCEXNlcnZlci5kYzEuY29uc3VsghVib2Iuc2VydmVyLmRj
MS5jb25zdWwwDQYJKoZIhvcNAQEFBQADggEBAGx4NH6cUIfLf4e/lvBDZFmd2qI9
+uYC0kjdbf8mZuyVvpbtaWHqVUdfGRXjYJUi6+T7MSzhx5hhtXEwkKRDQWO3DPkE
kOOh+NEfeWm0Qsz41TlEJmZnpZP4sF37qO8uquFL4gVO4fHlybjL43XoaUiGsJ6o
jDQWqPZTArUDKz3SfvRc00VLc2TQ0neLVcAl24m5t3MNaN1UZ4PI2cXfC6HtAiVz
9V7IgRtM38YTYe8MzkiXCwFUVubTSyOOexxtoY8TuYEvyGcUocsz+G+SzK3gieB7
D4MxZbgQzSOGtlDx9G7K5AWw+rqzReehDuzkI9itFXBAHKjudycE25a3xUQ=
-----END CERTIFICATE-----

7
test/hostname/Bob.ext Normal file
View File

@ -0,0 +1,7 @@
basicConstraints = CA:FALSE
keyUsage = nonRepudiation, digitalSignature, keyEncipherment
subjectAltName = @alt_names
[alt_names]
DNS.1 = server.dc1.consul
DNS.2 = bob.server.dc1.consul

28
test/hostname/Bob.key Normal file
View File

@ -0,0 +1,28 @@
-----BEGIN PRIVATE KEY-----
MIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQC+TMR+iyWgqvEm
aqDTS7AaK5M18oPF47dDPm/o6/RbPRDO1KfcXXaJCk14tTd2BbgUPHyuOf5CfEQI
Bc3JgI8Aj4nTY56Fo7Zz0igEOd2tXBe0scx0dXZPrRnnUfg1tG8kBJGYL4wR7Bd8
N0xCpZK4+6NWyEkGmiTCI+NoVevhadGDrTlLbs+1GvzuufUBOnVsam28beDfFI7J
oGFpV/wbu93C3BUs2yg7wvHrAw2uvA0K5A05Vk+w61gW9bKWHNGvOzTIr5ZWYFLY
O2xNq/9vbmnX/teYiMWd7OkZbwTssbV2L9NJ0hML7fd48Rb93jjXAXCqHQgliqUZ
45aTQEqlAgMBAAECggEAWrzeAHy2r1py699x2e5ABOp8IgAF5wjCbHTMBaLke9Ct
QAHUHFYQXB2mfQTjcgoeEMAarzSF0QvRoIWr7wW2qgzHKh1ZC93Y9Hbjj8hLtAqy
Xv1cQLd1d15ReKP0Fx920xS+m3Moda8+L4NqgGjUghGye4G6mERNfKiCGVDGzU7F
5ayIHR60BRiwsODJ7jr5ajcXoTHQ34gRLz/hB6S72sLAwEjGedpqpd79LNXkSdiP
axEW9nJVodc286WToR2YSSDezvIKgpZDy9onvBFmIyZIKuALmk10YNTrL1SfgR2C
wIjeHmfukgnlWzNFLB8bx0PBnaINSgxfdDa6ZYaaIQKBgQDmFWvmXUcW+SOidjUV
QTS5gjejYdDmB49y9x4eUffyHwA0wJWpiXE9TCy+PjLi1WIineHiaAmNngEU/IHF
NBi127opbU6CftvW7dGdv2IJxaN2IePSmlsLD8XItD+ZbhcZnHy4bLF8gIdttxXS
GZPHzesY0EqKCyb5ygjQ1wjZmQKBgQDTvCj6cLmVbV89wJMB2rSTglD9B2iwJnHX
wiX7bedc579odjUpTOmbPTxn9aI1MJeE9aKFuQP6NspOSXKQqlXjheXCs4d4jWmD
EQpL8dtHzXVdZf/2+RtuCYafpMRXFvraQjg5TdHT7ezQco74tW3CW2YUVdKyslNn
R1EWlzyY7QKBgQCotlyAdzWBqv5uSq9x/nZi8RFLRJahljmh24LCSOi/KexEwlL8
FkRq5kiI16MIod9r8smH8zHOHmY8tUuTBzh3Yb+IURaYqd0aJRjny0ZgVAQgw4kD
DRxlaBNnsIRSRV+67/ykX09mM/kagn4Fqaurf1s8vr9pqfPShksgmA1tQQKBgE98
lLmn9dOl8ppBIC8TBrVVt8e1r1RpqlVAOngQQ0n6aj3yGnT9vbkcnP++E/351vgA
KtoeoeKeDQakxhCPEZ1Pl/im4xWbqN+eVwo5qoNjG0tLznLOA8EkbFikR10WcGfd
cjP5BeuUp1F9oDS4D5NmMoUxzt5s2ais+kEL16DlAoGBAKoyjZDTv8mG0YCv4W92
Quv8+KxE5+7qGjckDZh1tZGQjU6br1QccPAXZmlRbAJD1c90uUO+Kkx27FFJrB4t
A9jCUpXUv78PyvqX3IUW8H555n/a0M37A0xnkqm91LddkKmAbkQvt6oel5rNbt2+
QeYzS1O8PX+zTLGf64h8Ajwt
-----END PRIVATE KEY-----

Some files were not shown because too many files have changed in this diff Show More