Warn if ACL is enabled but no token is provided to Envoy (#15967)

This commit is contained in:
Chris S. Kim 2023-01-16 12:31:56 -05:00 committed by GitHub
parent 9e99715ee8
commit c262065276
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 589 additions and 51 deletions

9
.changelog/15967.txt Normal file
View File

@ -0,0 +1,9 @@
```release-note:improvement
connect: Warn if ACLs are enabled but a token is not provided to envoy
```
```release-note:improvement
telemetry: Added a `consul.xds.server.streamsUnauthenticated` metric to track
the number of active xDS streams handled by the server that are unauthenticated
because ACLs are not enabled or ACL tokens were missing.
````

View File

@ -5,9 +5,10 @@ import (
"testing" "testing"
"time" "time"
"github.com/hashicorp/consul/agent/structs"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"google.golang.org/grpc/metadata" "google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/structs"
) )
func TestQueryOptionsFromContextRoundTrip(t *testing.T) { func TestQueryOptionsFromContextRoundTrip(t *testing.T) {

View File

@ -45,7 +45,7 @@ type ADSDeltaStream = envoy_discovery_v3.AggregatedDiscoveryService_DeltaAggrega
// DeltaAggregatedResources implements envoy_discovery_v3.AggregatedDiscoveryServiceServer // DeltaAggregatedResources implements envoy_discovery_v3.AggregatedDiscoveryServiceServer
func (s *Server) DeltaAggregatedResources(stream ADSDeltaStream) error { func (s *Server) DeltaAggregatedResources(stream ADSDeltaStream) error {
defer s.activeStreams.Increment("v3")() defer s.activeStreams.Increment(stream.Context())()
// a channel for receiving incoming requests // a channel for receiving incoming requests
reqCh := make(chan *envoy_discovery_v3.DeltaDiscoveryRequest) reqCh := make(chan *envoy_discovery_v3.DeltaDiscoveryRequest)

View File

@ -8,7 +8,6 @@ import (
"time" "time"
envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3" envoy_discovery_v3 "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
rpcstatus "google.golang.org/genproto/googleapis/rpc/status" rpcstatus "google.golang.org/genproto/googleapis/rpc/status"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
@ -1075,6 +1074,17 @@ func TestServer_DeltaAggregatedResources_v3_ACLEnforcement(t *testing.T) {
// includes the token in the ext rbac filter so lets us test more stuff. // includes the token in the ext rbac filter so lets us test more stuff.
envoy.SendDeltaReq(t, xdscommon.ListenerType, nil) envoy.SendDeltaReq(t, xdscommon.ListenerType, nil)
// If there is no token, check that we increment the gauge
if tt.token == "" {
data := scenario.sink.Data()
require.Len(t, data, 1)
item := data[0]
val, ok := item.Gauges["consul.xds.test.xds.server.streamsUnauthenticated"]
require.True(t, ok)
require.Equal(t, float32(1), val.Value)
}
if !tt.wantDenied { if !tt.wantDenied {
assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{ assertDeltaResponseSent(t, envoy.deltaStream.sendCh, &envoy_discovery_v3.DeltaDiscoveryResponse{
TypeUrl: xdscommon.ListenerType, TypeUrl: xdscommon.ListenerType,
@ -1106,6 +1116,17 @@ func TestServer_DeltaAggregatedResources_v3_ACLEnforcement(t *testing.T) {
case <-time.After(50 * time.Millisecond): case <-time.After(50 * time.Millisecond):
t.Fatalf("timed out waiting for handler to finish") t.Fatalf("timed out waiting for handler to finish")
} }
// If there is no token, check that we decrement the gauge
if tt.token == "" {
data := scenario.sink.Data()
require.Len(t, data, 1)
item := data[0]
val, ok := item.Gauges["consul.xds.test.xds.server.streamsUnauthenticated"]
require.True(t, ok)
require.Equal(t, float32(0), val.Value)
}
}) })
} }
} }
@ -1459,13 +1480,12 @@ func TestServer_DeltaAggregatedResources_v3_StreamDrained(t *testing.T) {
require.Len(t, data, 1) require.Len(t, data, 1)
item := data[0] item := data[0]
require.Len(t, item.Counters, 1) require.Len(t, item.Samples, 1)
val, ok := item.Samples["consul.xds.test.xds.server.streamStart"] val, ok := item.Samples["consul.xds.test.xds.server.streamStart"]
require.True(t, ok) require.True(t, ok)
require.Equal(t, 1, val.Count) require.Equal(t, 1, val.Count)
}) })
} }
type testLimiter struct { type testLimiter struct {

View File

@ -23,26 +23,30 @@ import (
"github.com/hashicorp/consul/agent/xds/xdscommon" "github.com/hashicorp/consul/agent/xds/xdscommon"
) )
var StatsGauges = []prometheus.GaugeDefinition{ var (
{ StatsGauges = []prometheus.GaugeDefinition{
Name: []string{"xds", "server", "streams"}, {
Help: "Measures the number of active xDS streams handled by the server split by protocol version.", Name: []string{"xds", "server", "streams"},
}, Help: "Measures the number of active xDS streams handled by the server split by protocol version.",
} },
{
var StatsCounters = []prometheus.CounterDefinition{ Name: []string{"xds", "server", "streamsUnauthenticated"},
{ Help: "Counts the number of active xDS streams handled by the server that are unauthenticated because ACLs are not enabled or ACL tokens were missing.",
Name: []string{"xds", "server", "streamDrained"}, },
Help: "Counts the number of xDS streams that are drained when rebalancing the load between servers.", }
}, StatsCounters = []prometheus.CounterDefinition{
} {
Name: []string{"xds", "server", "streamDrained"},
var StatsSummaries = []prometheus.SummaryDefinition{ Help: "Counts the number of xDS streams that are drained when rebalancing the load between servers.",
{ },
Name: []string{"xds", "server", "streamStart"}, }
Help: "Measures the time in milliseconds after an xDS stream is opened until xDS resources are first generated for the stream.", StatsSummaries = []prometheus.SummaryDefinition{
}, {
} Name: []string{"xds", "server", "streamStart"},
Help: "Measures the time in milliseconds after an xDS stream is opened until xDS resources are first generated for the stream.",
},
}
)
// ADSStream is a shorter way of referring to this thing... // ADSStream is a shorter way of referring to this thing...
type ADSStream = envoy_discovery_v3.AggregatedDiscoveryService_StreamAggregatedResourcesServer type ADSStream = envoy_discovery_v3.AggregatedDiscoveryService_StreamAggregatedResourcesServer
@ -126,32 +130,34 @@ type Server struct {
activeStreams *activeStreamCounters activeStreams *activeStreamCounters
} }
// activeStreamCounters simply encapsulates two counters accessed atomically to // activeStreamCounters tracks various stream-related metrics.
// ensure alignment is correct. This further requires that activeStreamCounters // Requires that activeStreamCounters be a pointer field.
// be a pointer field.
// TODO(eculver): refactor to remove xDSv2 refs
type activeStreamCounters struct { type activeStreamCounters struct {
xDSv3 uint64 xDSv3 atomic.Uint64
xDSv2 uint64 unauthenticated atomic.Uint64
} }
func (c *activeStreamCounters) Increment(xdsVersion string) func() { func (c *activeStreamCounters) Increment(ctx context.Context) func() {
var counter *uint64 // If no ACL token is found, increase the gauge.
switch xdsVersion { o, _ := external.QueryOptionsFromContext(ctx)
case "v3": if o.Token == "" {
counter = &c.xDSv3 unauthn := c.unauthenticated.Add(1)
case "v2": metrics.SetGauge([]string{"xds", "server", "streamsUnauthenticated"}, float32(unauthn))
counter = &c.xDSv2
default:
return func() {}
} }
labels := []metrics.Label{{Name: "version", Value: xdsVersion}} // Historically there had been a "v2" version.
labels := []metrics.Label{{Name: "version", Value: "v3"}}
count := atomic.AddUint64(counter, 1) count := c.xDSv3.Add(1)
metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels) metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels)
// This closure should be called in a defer to decrement the gauges after the stream is closed.
return func() { return func() {
count := atomic.AddUint64(counter, ^uint64(0)) if o.Token == "" {
unauthn := c.unauthenticated.Add(^uint64(0))
metrics.SetGauge([]string{"xds", "server", "streamsUnauthenticated"}, float32(unauthn))
}
count := c.xDSv3.Add(^uint64(0))
metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels) metrics.SetGaugeWithLabels([]string{"xds", "server", "streams"}, float32(count), labels)
} }
} }

View File

@ -810,7 +810,7 @@ func requireProtocolVersionGauge(
require.Len(t, data, 1) require.Len(t, data, 1)
item := data[0] item := data[0]
require.Len(t, item.Gauges, 1) require.Len(t, item.Gauges, 2)
val, ok := item.Gauges["consul.xds.test.xds.server.streams;version="+xdsVersion] val, ok := item.Gauges["consul.xds.test.xds.server.streams;version="+xdsVersion]
require.True(t, ok) require.True(t, ok)

View File

@ -16,6 +16,7 @@ import (
"github.com/mitchellh/mapstructure" "github.com/mitchellh/mapstructure"
"google.golang.org/protobuf/encoding/protojson" "google.golang.org/protobuf/encoding/protojson"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/xds" "github.com/hashicorp/consul/agent/xds"
"github.com/hashicorp/consul/agent/xds/accesslogs" "github.com/hashicorp/consul/agent/xds/accesslogs"
@ -612,6 +613,16 @@ func (c *cmd) generateConfig() ([]byte, error) {
var bsCfg BootstrapConfig var bsCfg BootstrapConfig
// Make a call to an arbitrary ACL endpoint. If we get back an ErrNotFound
// (meaning ACLs are enabled) check that the token is not empty.
if _, _, err := c.client.ACL().TokenReadSelf(
&api.QueryOptions{Token: args.Token},
); acl.IsErrNotFound(err) {
if args.Token == "" {
c.UI.Warn("No ACL token was provided to Envoy. Because the ACL system is enabled, pass a suitable ACL token for this service to Envoy to avoid potential communication failure.")
}
}
// Fetch any customization from the registration // Fetch any customization from the registration
var svcProxyConfig *api.AgentServiceConnectProxyConfig var svcProxyConfig *api.AgentServiceConnectProxyConfig
var serviceName, ns, partition, datacenter string var serviceName, ns, partition, datacenter string
@ -754,7 +765,6 @@ func generateAccessLogs(c *cmd, args *BootstrapTplArgs) error {
return nil return nil
} }
// TODO: make method a function
func (c *cmd) xdsAddress() (GRPC, error) { func (c *cmd) xdsAddress() (GRPC, error) {
g := GRPC{} g := GRPC{}

View File

@ -17,6 +17,7 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent" "github.com/hashicorp/consul/agent"
"github.com/hashicorp/consul/agent/xds" "github.com/hashicorp/consul/agent/xds"
"github.com/hashicorp/consul/agent/xds/proxysupport" "github.com/hashicorp/consul/agent/xds/proxysupport"
@ -113,6 +114,7 @@ func testSetAndResetEnv(t *testing.T, env []string) func() {
type generateConfigTestCase struct { type generateConfigTestCase struct {
Name string Name string
TLSServer bool TLSServer bool
ACLEnabled bool
Flags []string Flags []string
Env []string Env []string
Files map[string]string Files map[string]string
@ -123,6 +125,7 @@ type generateConfigTestCase struct {
AgentSelf110 bool // fake the agent API from versions v1.10 and earlier AgentSelf110 bool // fake the agent API from versions v1.10 and earlier
WantArgs BootstrapTplArgs WantArgs BootstrapTplArgs
WantErr string WantErr string
WantWarn string
} }
// This tests the args we use to generate the template directly because they // This tests the args we use to generate the template directly because they
@ -553,8 +556,9 @@ func TestGenerateConfig(t *testing.T) {
}, },
}, },
{ {
Name: "access-log-path", Name: "access-log-path",
Flags: []string{"-proxy-id", "test-proxy", "-admin-access-log-path", "/some/path/access.log"}, Flags: []string{"-proxy-id", "test-proxy", "-admin-access-log-path", "/some/path/access.log"},
WantWarn: "-admin-access-log-path is deprecated",
WantArgs: BootstrapTplArgs{ WantArgs: BootstrapTplArgs{
ProxyCluster: "test-proxy", ProxyCluster: "test-proxy",
ProxyID: "test-proxy", ProxyID: "test-proxy",
@ -1116,6 +1120,52 @@ func TestGenerateConfig(t *testing.T) {
}, },
}, },
}, },
{
Name: "acl-enabled-but-no-token",
Flags: []string{"-proxy-id", "test-proxy"},
ACLEnabled: true,
WantWarn: "No ACL token was provided to Envoy.",
WantArgs: BootstrapTplArgs{
ProxyCluster: "test-proxy",
ProxyID: "test-proxy",
// We don't know this til after the lookup so it will be empty in the
// initial args call we are testing here.
ProxySourceService: "",
GRPC: GRPC{
AgentAddress: "127.0.0.1",
AgentPort: "8502", // Note this is the gRPC port
},
AdminAccessLogPath: "/dev/null",
AdminBindAddress: "127.0.0.1",
AdminBindPort: "19000",
LocalAgentClusterName: xds.LocalAgentClusterName,
PrometheusBackendPort: "",
PrometheusScrapePath: "/metrics",
},
},
{
Name: "acl-enabled-and-token",
Flags: []string{"-proxy-id", "test-proxy", "-token", "foo"},
ACLEnabled: true,
WantArgs: BootstrapTplArgs{
ProxyCluster: "test-proxy",
ProxyID: "test-proxy",
// We don't know this til after the lookup so it will be empty in the
// initial args call we are testing here.
ProxySourceService: "",
GRPC: GRPC{
AgentAddress: "127.0.0.1",
AgentPort: "8502", // Note this is the gRPC port
},
AdminAccessLogPath: "/dev/null",
AdminBindAddress: "127.0.0.1",
AdminBindPort: "19000",
Token: "foo",
LocalAgentClusterName: xds.LocalAgentClusterName,
PrometheusBackendPort: "",
PrometheusScrapePath: "/metrics",
},
},
} }
cases = append(cases, enterpriseGenerateConfigTestCases()...) cases = append(cases, enterpriseGenerateConfigTestCases()...)
@ -1177,12 +1227,16 @@ func TestGenerateConfig(t *testing.T) {
require.NoError(t, c.flags.Parse(args)) require.NoError(t, c.flags.Parse(args))
code := c.run(c.flags.Args()) code := c.run(c.flags.Args())
if tc.WantErr == "" { if tc.WantErr != "" {
require.Equal(t, 0, code, ui.ErrorWriter.String())
} else {
require.Equal(t, 1, code, ui.ErrorWriter.String()) require.Equal(t, 1, code, ui.ErrorWriter.String())
require.Contains(t, ui.ErrorWriter.String(), tc.WantErr) require.Contains(t, ui.ErrorWriter.String(), tc.WantErr)
return return
} else if tc.WantWarn != "" {
require.Equal(t, 0, code, ui.ErrorWriter.String())
require.Contains(t, ui.ErrorWriter.String(), tc.WantWarn)
} else {
require.Equal(t, 0, code, ui.ErrorWriter.String())
require.Empty(t, ui.ErrorWriter.String())
} }
// Verify we handled the env and flags right first to get correct template // Verify we handled the env and flags right first to get correct template
@ -1316,6 +1370,8 @@ func testMockAgent(tc generateConfigTestCase) http.HandlerFunc {
testMockCatalogNodeServiceList()(w, r) testMockCatalogNodeServiceList()(w, r)
case strings.Contains(r.URL.Path, "/config/proxy-defaults/global"): case strings.Contains(r.URL.Path, "/config/proxy-defaults/global"):
testMockConfigProxyDefaults(tc.ProxyDefaults)(w, r) testMockConfigProxyDefaults(tc.ProxyDefaults)(w, r)
case strings.Contains(r.URL.Path, "/acl/token/self"):
testMockTokenReadSelf(tc.ACLEnabled, tc.Flags)(w, r)
default: default:
http.NotFound(w, r) http.NotFound(w, r)
} }
@ -1467,6 +1523,21 @@ func testMockConfigProxyDefaults(entry api.ProxyConfigEntry) http.HandlerFunc {
} }
} }
func testMockTokenReadSelf(aclEnabled bool, flags []string) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) {
if aclEnabled {
for _, f := range flags {
if f == "-token" {
w.WriteHeader(200)
return
}
}
w.WriteHeader(403)
w.Write([]byte(acl.ErrNotFound.Error()))
return
}
}
}
func TestEnvoyCommand_canBindInternal(t *testing.T) { func TestEnvoyCommand_canBindInternal(t *testing.T) {
t.Parallel() t.Parallel()
type testCheck struct { type testCheck struct {

View File

@ -0,0 +1,210 @@
{
"admin": {
"access_log_path": "/dev/null",
"address": {
"socket_address": {
"address": "127.0.0.1",
"port_value": 19000
}
}
},
"node": {
"cluster": "test",
"id": "test-proxy",
"metadata": {
"namespace": "default",
"partition": "default"
}
},
"layered_runtime": {
"layers": [
{
"name": "base",
"static_layer": {
"re2.max_program_size.error_level": 1048576
}
}
]
},
"static_resources": {
"clusters": [
{
"name": "local_agent",
"ignore_health_on_host_removal": false,
"connect_timeout": "1s",
"type": "STATIC",
"http2_protocol_options": {},
"loadAssignment": {
"clusterName": "local_agent",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socket_address": {
"address": "127.0.0.1",
"port_value": 8502
}
}
}
}
]
}
]
}
}
]
},
"stats_config": {
"stats_tags": [
{
"regex": "^cluster\\.(?:passthrough~)?((?:([^.]+)~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.custom_hash"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:([^.]+)\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.service_subset"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?([^.]+)\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.service"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.namespace"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:([^.]+)\\.)?[^.]+\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.partition"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?([^.]+)\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.datacenter"
},
{
"regex": "^cluster\\.([^.]+\\.(?:[^.]+\\.)?([^.]+)\\.external\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.peer"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.routing_type"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.([^.]+)\\.consul\\.)",
"tag_name": "consul.destination.trust_domain"
},
{
"regex": "^cluster\\.(?:passthrough~)?(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+)\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.target"
},
{
"regex": "^cluster\\.(?:passthrough~)?(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+)\\.consul\\.)",
"tag_name": "consul.destination.full_target"
},
{
"regex": "^(?:tcp|http)\\.upstream(?:_peered)?\\.(([^.]+)(?:\\.[^.]+)?(?:\\.[^.]+)?\\.[^.]+\\.)",
"tag_name": "consul.upstream.service"
},
{
"regex": "^(?:tcp|http)\\.upstream\\.([^.]+(?:\\.[^.]+)?(?:\\.[^.]+)?\\.([^.]+)\\.)",
"tag_name": "consul.upstream.datacenter"
},
{
"regex": "^(?:tcp|http)\\.upstream_peered\\.([^.]+(?:\\.[^.]+)?\\.([^.]+)\\.)",
"tag_name": "consul.upstream.peer"
},
{
"regex": "^(?:tcp|http)\\.upstream(?:_peered)?\\.([^.]+(?:\\.([^.]+))?(?:\\.[^.]+)?\\.[^.]+\\.)",
"tag_name": "consul.upstream.namespace"
},
{
"regex": "^(?:tcp|http)\\.upstream\\.([^.]+(?:\\.[^.]+)?(?:\\.([^.]+))?\\.[^.]+\\.)",
"tag_name": "consul.upstream.partition"
},
{
"regex": "^cluster\\.((?:([^.]+)~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.custom_hash"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:([^.]+)\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.service_subset"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?([^.]+)\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.service"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.namespace"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?([^.]+)\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.datacenter"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.[^.]+\\.consul\\.)",
"tag_name": "consul.routing_type"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.([^.]+)\\.consul\\.)",
"tag_name": "consul.trust_domain"
},
{
"regex": "^cluster\\.(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+)\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.target"
},
{
"regex": "^cluster\\.(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+)\\.consul\\.)",
"tag_name": "consul.full_target"
},
{
"tag_name": "local_cluster",
"fixed_value": "test"
},
{
"tag_name": "consul.source.service",
"fixed_value": "test"
},
{
"tag_name": "consul.source.namespace",
"fixed_value": "default"
},
{
"tag_name": "consul.source.partition",
"fixed_value": "default"
},
{
"tag_name": "consul.source.datacenter",
"fixed_value": "dc1"
}
],
"use_all_default_tags": true
},
"dynamic_resources": {
"lds_config": {
"ads": {},
"resource_api_version": "V3"
},
"cds_config": {
"ads": {},
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [
{
"key": "x-consul-token",
"value": "foo"
}
],
"envoy_grpc": {
"cluster_name": "local_agent"
}
}
}
}
}

View File

@ -0,0 +1,210 @@
{
"admin": {
"access_log_path": "/dev/null",
"address": {
"socket_address": {
"address": "127.0.0.1",
"port_value": 19000
}
}
},
"node": {
"cluster": "test",
"id": "test-proxy",
"metadata": {
"namespace": "default",
"partition": "default"
}
},
"layered_runtime": {
"layers": [
{
"name": "base",
"static_layer": {
"re2.max_program_size.error_level": 1048576
}
}
]
},
"static_resources": {
"clusters": [
{
"name": "local_agent",
"ignore_health_on_host_removal": false,
"connect_timeout": "1s",
"type": "STATIC",
"http2_protocol_options": {},
"loadAssignment": {
"clusterName": "local_agent",
"endpoints": [
{
"lbEndpoints": [
{
"endpoint": {
"address": {
"socket_address": {
"address": "127.0.0.1",
"port_value": 8502
}
}
}
}
]
}
]
}
}
]
},
"stats_config": {
"stats_tags": [
{
"regex": "^cluster\\.(?:passthrough~)?((?:([^.]+)~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.custom_hash"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:([^.]+)\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.service_subset"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?([^.]+)\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.service"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.namespace"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:([^.]+)\\.)?[^.]+\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.partition"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?([^.]+)\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.datacenter"
},
{
"regex": "^cluster\\.([^.]+\\.(?:[^.]+\\.)?([^.]+)\\.external\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.peer"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.routing_type"
},
{
"regex": "^cluster\\.(?:passthrough~)?((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.([^.]+)\\.consul\\.)",
"tag_name": "consul.destination.trust_domain"
},
{
"regex": "^cluster\\.(?:passthrough~)?(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+)\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.destination.target"
},
{
"regex": "^cluster\\.(?:passthrough~)?(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+)\\.consul\\.)",
"tag_name": "consul.destination.full_target"
},
{
"regex": "^(?:tcp|http)\\.upstream(?:_peered)?\\.(([^.]+)(?:\\.[^.]+)?(?:\\.[^.]+)?\\.[^.]+\\.)",
"tag_name": "consul.upstream.service"
},
{
"regex": "^(?:tcp|http)\\.upstream\\.([^.]+(?:\\.[^.]+)?(?:\\.[^.]+)?\\.([^.]+)\\.)",
"tag_name": "consul.upstream.datacenter"
},
{
"regex": "^(?:tcp|http)\\.upstream_peered\\.([^.]+(?:\\.[^.]+)?\\.([^.]+)\\.)",
"tag_name": "consul.upstream.peer"
},
{
"regex": "^(?:tcp|http)\\.upstream(?:_peered)?\\.([^.]+(?:\\.([^.]+))?(?:\\.[^.]+)?\\.[^.]+\\.)",
"tag_name": "consul.upstream.namespace"
},
{
"regex": "^(?:tcp|http)\\.upstream\\.([^.]+(?:\\.[^.]+)?(?:\\.([^.]+))?\\.[^.]+\\.)",
"tag_name": "consul.upstream.partition"
},
{
"regex": "^cluster\\.((?:([^.]+)~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.custom_hash"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:([^.]+)\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.service_subset"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?([^.]+)\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.service"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.namespace"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?([^.]+)\\.internal[^.]*\\.[^.]+\\.consul\\.)",
"tag_name": "consul.datacenter"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.([^.]+)\\.[^.]+\\.consul\\.)",
"tag_name": "consul.routing_type"
},
{
"regex": "^cluster\\.((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.([^.]+)\\.consul\\.)",
"tag_name": "consul.trust_domain"
},
{
"regex": "^cluster\\.(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+)\\.[^.]+\\.[^.]+\\.consul\\.)",
"tag_name": "consul.target"
},
{
"regex": "^cluster\\.(((?:[^.]+~)?(?:[^.]+\\.)?[^.]+\\.[^.]+\\.(?:[^.]+\\.)?[^.]+\\.[^.]+\\.[^.]+)\\.consul\\.)",
"tag_name": "consul.full_target"
},
{
"tag_name": "local_cluster",
"fixed_value": "test"
},
{
"tag_name": "consul.source.service",
"fixed_value": "test"
},
{
"tag_name": "consul.source.namespace",
"fixed_value": "default"
},
{
"tag_name": "consul.source.partition",
"fixed_value": "default"
},
{
"tag_name": "consul.source.datacenter",
"fixed_value": "dc1"
}
],
"use_all_default_tags": true
},
"dynamic_resources": {
"lds_config": {
"ads": {},
"resource_api_version": "V3"
},
"cds_config": {
"ads": {},
"resource_api_version": "V3"
},
"ads_config": {
"api_type": "DELTA_GRPC",
"transport_api_version": "V3",
"grpc_services": {
"initial_metadata": [
{
"key": "x-consul-token",
"value": ""
}
],
"envoy_grpc": {
"cluster_name": "local_agent"
}
}
}
}
}

View File

@ -541,6 +541,7 @@ These metrics are used to monitor the health of the Consul servers.
| `consul.grpc.server.stream.count` | Counts the number of new gRPC streams received by the server. Includes a `server_type` label indicating either the `internal` or `external` gRPC server. | streams | counter | | `consul.grpc.server.stream.count` | Counts the number of new gRPC streams received by the server. Includes a `server_type` label indicating either the `internal` or `external` gRPC server. | streams | counter |
| `consul.grpc.server.streams` | Measures the number of active gRPC streams handled by the server. Includes a `server_type` label indicating either the `internal` or `external` gRPC server. | streams | gauge | | `consul.grpc.server.streams` | Measures the number of active gRPC streams handled by the server. Includes a `server_type` label indicating either the `internal` or `external` gRPC server. | streams | gauge |
| `consul.xds.server.streams` | Measures the number of active xDS streams handled by the server split by protocol version. | streams | gauge | | `consul.xds.server.streams` | Measures the number of active xDS streams handled by the server split by protocol version. | streams | gauge |
| `consul.xds.server.streamsUnauthenticated` | Measures the number of active xDS streams handled by the server that are unauthenticated because ACLs are not enabled or ACL tokens were missing. | streams | gauge |
| `consul.xds.server.idealStreamsMax` | The maximum number of xDS streams per server, chosen to achieve a roughly even spread of load across servers. | streams | gauge | | `consul.xds.server.idealStreamsMax` | The maximum number of xDS streams per server, chosen to achieve a roughly even spread of load across servers. | streams | gauge |
| `consul.xds.server.streamDrained` | Counts the number of xDS streams that are drained when rebalancing the load between servers. | streams | counter | | `consul.xds.server.streamDrained` | Counts the number of xDS streams that are drained when rebalancing the load between servers. | streams | counter |
| `consul.xds.server.streamStart` | Measures the time taken to first generate xDS resources after an xDS stream is opened. | ms | timer | | `consul.xds.server.streamStart` | Measures the time taken to first generate xDS resources after an xDS stream is opened. | ms | timer |