Added OpenTelemetry Access Logging Envoy extension (#18336)

This commit is contained in:
Hardik Shingala 2023-08-30 20:21:58 +05:30 committed by GitHub
parent f8812eddf1
commit 58e5658810
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 1409 additions and 12 deletions

7
.changelog/18336.txt Normal file
View File

@ -0,0 +1,7 @@
```release-note:feature
xds: Add a built-in Envoy extension that appends OpenTelemetry Access Logging (otel-access-logging) to the HTTP Connection Manager filter.
```
```release-note:feature
xds: Add support for patching outbound listeners to the built-in Envoy External Authorization extension.
```

View File

@ -23,6 +23,8 @@ type extAuthz struct {
ProxyType api.ServiceKind
// InsertOptions controls how the extension inserts the filter.
InsertOptions ext_cmn.InsertOptions
// ListenerType controls which listener the extension applies to. It supports "inbound" or "outbound" listeners.
ListenerType string
// Config holds the extension configuration.
Config extAuthzConfig
}
@ -61,10 +63,14 @@ func (a *extAuthz) PatchClusters(cfg *ext_cmn.RuntimeConfig, c ext_cmn.ClusterMa
return c, nil
}
func (a *extAuthz) matchesListenerDirection(isInboundListener bool) bool {
return (!isInboundListener && a.ListenerType == "outbound") || (isInboundListener && a.ListenerType == "inbound")
}
// PatchFilters inserts an ext-authz filter into the list of network filters or the filter chain of the HTTP connection manager.
func (a *extAuthz) PatchFilters(cfg *ext_cmn.RuntimeConfig, filters []*envoy_listener_v3.Filter, isInboundListener bool) ([]*envoy_listener_v3.Filter, error) {
// The ext_authz extension only patches filters for inbound listeners.
if !isInboundListener {
if !a.matchesListenerDirection(isInboundListener) {
return filters, nil
}
@ -129,6 +135,11 @@ func (a *extAuthz) normalize() {
if a.ProxyType == "" {
a.ProxyType = api.ServiceKindConnectProxy
}
if a.ListenerType == "" {
a.ListenerType = "inbound"
}
a.Config.normalize()
}
@ -140,6 +151,10 @@ func (a *extAuthz) validate() error {
api.ServiceKindConnectProxy))
}
if a.ListenerType != "inbound" && a.ListenerType != "outbound" {
resultErr = multierror.Append(resultErr, fmt.Errorf(`unexpected ListenerType %q, supported values are "inbound" or "outbound"`, a.ListenerType))
}
if err := a.Config.validate(); err != nil {
resultErr = multierror.Append(resultErr, err)
}

View File

@ -0,0 +1,274 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package otelaccesslogging
import (
"fmt"
envoy_extensions_access_loggers_v3 "github.com/envoyproxy/go-control-plane/envoy/config/accesslog/v3"
envoy_listener_v3 "github.com/envoyproxy/go-control-plane/envoy/config/listener/v3"
envoy_extensions_access_loggers_otel_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/access_loggers/open_telemetry/v3"
"github.com/mitchellh/mapstructure"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/types/known/anypb"
"github.com/hashicorp/consul/api"
ext_cmn "github.com/hashicorp/consul/envoyextensions/extensioncommon"
"github.com/hashicorp/go-multierror"
v1 "go.opentelemetry.io/proto/otlp/common/v1"
)
type otelAccessLogging struct {
ext_cmn.BasicExtensionAdapter
// ProxyType identifies the type of Envoy proxy that this extension applies to.
// The extension will only be configured for proxies that match this type and
// will be ignored for all other proxy types.
ProxyType api.ServiceKind
// ListenerType controls which listener the extension applies to. It supports "inbound" or "outbound" listeners.
ListenerType string
// Config holds the extension configuration.
Config AccessLog
}
var _ ext_cmn.BasicExtension = (*otelAccessLogging)(nil)
func Constructor(ext api.EnvoyExtension) (ext_cmn.EnvoyExtender, error) {
otel, err := newOTELAccessLogging(ext)
if err != nil {
return nil, err
}
return &ext_cmn.BasicEnvoyExtender{
Extension: otel,
}, nil
}
// CanApply indicates if the extension can be applied to the given extension runtime configuration.
func (a *otelAccessLogging) CanApply(config *ext_cmn.RuntimeConfig) bool {
return config.Kind == api.ServiceKindConnectProxy
}
// PatchClusters modifies the cluster resources for the extension.
//
// If the extension is configured to target the OTEL service running on the local host network
// this func will insert a cluster for calling that service. It does nothing if the extension is
// configured to target an upstream service because the existing cluster for the upstream will be
// used directly by the filter.
func (a *otelAccessLogging) PatchClusters(cfg *ext_cmn.RuntimeConfig, c ext_cmn.ClusterMap) (ext_cmn.ClusterMap, error) {
cluster, err := a.Config.toEnvoyCluster(cfg)
if err != nil {
return c, err
}
if cluster != nil {
c[cluster.Name] = cluster
}
return c, nil
}
func (a *otelAccessLogging) matchesListenerDirection(p ext_cmn.FilterPayload) bool {
isInboundListener := p.IsInbound()
return (!isInboundListener && a.ListenerType == "outbound") || (isInboundListener && a.ListenerType == "inbound")
}
// PatchFilter adds the OTEL access log in the HTTP connection manager.
func (a *otelAccessLogging) PatchFilter(p ext_cmn.FilterPayload) (*envoy_listener_v3.Filter, bool, error) {
filter := p.Message
// Make sure filter matches extension config.
if !a.matchesListenerDirection(p) {
return filter, false, nil
}
httpConnectionManager, _, err := ext_cmn.GetHTTPConnectionManager(filter)
if err != nil {
return filter, false, err
}
accessLog, err := a.toEnvoyAccessLog(p.RuntimeConfig)
if err != nil {
return filter, false, err
}
httpConnectionManager.AccessLog = append(httpConnectionManager.AccessLog, accessLog)
newHCM, err := ext_cmn.MakeFilter("envoy.filters.network.http_connection_manager", httpConnectionManager)
if err != nil {
return filter, false, err
}
return newHCM, true, nil
}
func newOTELAccessLogging(ext api.EnvoyExtension) (*otelAccessLogging, error) {
otel := &otelAccessLogging{}
if ext.Name != api.BuiltinOTELAccessLoggingExtension {
return otel, fmt.Errorf("expected extension name %q but got %q", api.BuiltinOTELAccessLoggingExtension, ext.Name)
}
if err := otel.fromArguments(ext.Arguments); err != nil {
return otel, err
}
return otel, nil
}
func (a *otelAccessLogging) fromArguments(args map[string]any) error {
if err := mapstructure.Decode(args, a); err != nil {
return err
}
a.normalize()
return a.validate()
}
func (a *otelAccessLogging) toEnvoyAccessLog(cfg *ext_cmn.RuntimeConfig) (*envoy_extensions_access_loggers_v3.AccessLog, error) {
commonConfig, err := a.Config.toEnvoyCommonGrpcAccessLogConfig(cfg)
if err != nil {
return nil, err
}
body, err := toEnvoyAnyValue(a.Config.Body)
if err != nil {
return nil, fmt.Errorf("failed to marshal Body: %w", err)
}
attributes, err := toEnvoyKeyValueList(a.Config.Attributes)
if err != nil {
return nil, fmt.Errorf("failed to marshal Attributes: %w", err)
}
resourceAttributes, err := toEnvoyKeyValueList(a.Config.ResourceAttributes)
if err != nil {
return nil, fmt.Errorf("failed to marshal ResourceAttributes: %w", err)
}
otelAccessLogConfig := &envoy_extensions_access_loggers_otel_v3.OpenTelemetryAccessLogConfig{
CommonConfig: commonConfig,
Body: body,
Attributes: attributes,
ResourceAttributes: resourceAttributes,
}
// Marshal the struct to bytes.
otelAccessLogConfigBytes, err := proto.Marshal(otelAccessLogConfig)
if err != nil {
return nil, fmt.Errorf("failed to marshal OpenTelemetryAccessLogConfig: %w", err)
}
return &envoy_extensions_access_loggers_v3.AccessLog{
Name: "envoy.access_loggers.open_telemetry",
ConfigType: &envoy_extensions_access_loggers_v3.AccessLog_TypedConfig{
TypedConfig: &anypb.Any{
Value: otelAccessLogConfigBytes,
TypeUrl: "type.googleapis.com/envoy.extensions.access_loggers.open_telemetry.v3.OpenTelemetryAccessLogConfig",
},
},
}, nil
}
func (a *otelAccessLogging) normalize() {
if a.ProxyType == "" {
a.ProxyType = api.ServiceKindConnectProxy
}
if a.ListenerType == "" {
a.ListenerType = "inbound"
}
if a.Config.LogName == "" {
a.Config.LogName = a.ListenerType
}
a.Config.normalize()
}
func (a *otelAccessLogging) validate() error {
var resultErr error
if a.ProxyType != api.ServiceKindConnectProxy {
resultErr = multierror.Append(resultErr, fmt.Errorf("unsupported ProxyType %q, only %q is supported",
a.ProxyType,
api.ServiceKindConnectProxy))
}
if a.ListenerType != "inbound" && a.ListenerType != "outbound" {
resultErr = multierror.Append(resultErr, fmt.Errorf(`unexpected ListenerType %q, supported values are "inbound" or "outbound"`, a.ListenerType))
}
if err := a.Config.validate(); err != nil {
resultErr = multierror.Append(resultErr, err)
}
return resultErr
}
func toEnvoyKeyValueList(attributes map[string]any) (*v1.KeyValueList, error) {
keyValueList := &v1.KeyValueList{}
for key, value := range attributes {
anyValue, err := toEnvoyAnyValue(value)
if err != nil {
return nil, err
}
keyValueList.Values = append(keyValueList.Values, &v1.KeyValue{
Key: key,
Value: anyValue,
})
}
return keyValueList, nil
}
func toEnvoyAnyValue(value interface{}) (*v1.AnyValue, error) {
if value == nil {
return nil, nil
}
switch v := value.(type) {
case string:
return &v1.AnyValue{
Value: &v1.AnyValue_StringValue{
StringValue: v,
},
}, nil
case int:
return &v1.AnyValue{
Value: &v1.AnyValue_IntValue{
IntValue: int64(v),
},
}, nil
case int32:
return &v1.AnyValue{
Value: &v1.AnyValue_IntValue{
IntValue: int64(v),
},
}, nil
case int64:
return &v1.AnyValue{
Value: &v1.AnyValue_IntValue{
IntValue: v,
},
}, nil
case float32:
return &v1.AnyValue{
Value: &v1.AnyValue_DoubleValue{
DoubleValue: float64(v),
},
}, nil
case float64:
return &v1.AnyValue{
Value: &v1.AnyValue_DoubleValue{
DoubleValue: v,
},
}, nil
case bool:
return &v1.AnyValue{
Value: &v1.AnyValue_BoolValue{
BoolValue: v,
},
}, nil
case []byte:
return &v1.AnyValue{
Value: &v1.AnyValue_BytesValue{
BytesValue: v,
},
}, nil
default:
return nil, fmt.Errorf("unsupported type %T", v)
}
}

View File

@ -0,0 +1,113 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package otelaccesslogging
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/envoyextensions/extensioncommon"
)
func TestConstructor(t *testing.T) {
makeArguments := func(overrides map[string]interface{}) map[string]interface{} {
m := map[string]interface{}{
"ProxyType": "connect-proxy",
"ListenerType": "inbound",
"Config": AccessLog{
LogName: "access.log",
GrpcService: &GrpcService{
Target: &Target{
Service: api.CompoundServiceName{
Name: "otel-collector",
Namespace: "default",
Partition: "default",
},
},
},
},
}
for k, v := range overrides {
m[k] = v
}
return m
}
cases := map[string]struct {
extensionName string
arguments map[string]interface{}
expected otelAccessLogging
ok bool
}{
"with no arguments": {
arguments: nil,
ok: false,
},
"with an invalid name": {
arguments: makeArguments(map[string]interface{}{}),
extensionName: "bad",
ok: false,
},
"invalid proxy type": {
arguments: makeArguments(map[string]interface{}{"ProxyType": "terminating-gateway"}),
ok: false,
},
"invalid listener": {
arguments: makeArguments(map[string]interface{}{"ListenerType": "invalid"}),
ok: false,
},
"default proxy type": {
arguments: makeArguments(map[string]interface{}{"ProxyType": ""}),
expected: otelAccessLogging{
ProxyType: "connect-proxy",
ListenerType: "inbound",
Config: AccessLog{
LogName: "access.log",
GrpcService: &GrpcService{
Target: &Target{
Service: api.CompoundServiceName{
Name: "otel-collector",
Namespace: "default",
Partition: "default",
},
},
},
},
},
ok: true,
},
}
for n, tc := range cases {
t.Run(n, func(t *testing.T) {
extensionName := api.BuiltinOTELAccessLoggingExtension
if tc.extensionName != "" {
extensionName = tc.extensionName
}
svc := api.CompoundServiceName{Name: "svc"}
ext := extensioncommon.RuntimeConfig{
ServiceName: svc,
EnvoyExtension: api.EnvoyExtension{
Name: extensionName,
Arguments: tc.arguments,
},
}
e, err := Constructor(ext.EnvoyExtension)
if tc.ok {
require.NoError(t, err)
require.Equal(t, &extensioncommon.BasicEnvoyExtender{Extension: &tc.expected}, e)
} else {
require.Error(t, err)
}
})
}
}

View File

@ -0,0 +1,424 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
package otelaccesslogging
import (
"fmt"
"strconv"
"strings"
"time"
envoy_cluster_v3 "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3"
envoy_core_v3 "github.com/envoyproxy/go-control-plane/envoy/config/core/v3"
envoy_endpoint_v3 "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3"
envoy_extensions_access_loggers_grpc_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/access_loggers/grpc/v3"
envoy_upstreams_http_v3 "github.com/envoyproxy/go-control-plane/envoy/extensions/upstreams/http/v3"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/api"
cmn "github.com/hashicorp/consul/envoyextensions/extensioncommon"
"github.com/hashicorp/go-multierror"
"google.golang.org/protobuf/types/known/anypb"
"google.golang.org/protobuf/types/known/durationpb"
"google.golang.org/protobuf/types/known/wrapperspb"
)
const (
LocalAccessLogClusterName = "local_access_log"
localhost = "localhost"
localhostIPv4 = "127.0.0.1"
localhostIPv6 = "::1"
)
type AccessLog struct {
LogName string
GrpcService *GrpcService
BufferFlushInterval *time.Duration
BufferSizeBytes uint32
FilterStateObjectsToLog []string
RetryPolicy *RetryPolicy
Body interface{}
Attributes map[string]interface{}
ResourceAttributes map[string]interface{}
}
func (a *AccessLog) normalize() {
if a.GrpcService != nil {
a.GrpcService.normalize()
}
if a.RetryPolicy != nil {
a.RetryPolicy.normalize()
}
}
func (a *AccessLog) validate() error {
a.normalize()
if a.GrpcService == nil {
return fmt.Errorf("missing GrpcService")
}
var resultErr error
var field string
var validate func() error
field = "GrpcService"
validate = a.GrpcService.validate
if err := validate(); err != nil {
resultErr = multierror.Append(resultErr, fmt.Errorf("failed to validate Config.%s: %w", field, err))
}
return resultErr
}
func (a *AccessLog) envoyGrpcService(cfg *cmn.RuntimeConfig) (*envoy_core_v3.GrpcService, error) {
target := a.GrpcService.Target
clusterName, err := a.getClusterName(cfg, target)
if err != nil {
return nil, err
}
var initialMetadata []*envoy_core_v3.HeaderValue
for _, meta := range a.GrpcService.InitialMetadata {
initialMetadata = append(initialMetadata, meta.toEnvoy())
}
return &envoy_core_v3.GrpcService{
TargetSpecifier: &envoy_core_v3.GrpcService_EnvoyGrpc_{
EnvoyGrpc: &envoy_core_v3.GrpcService_EnvoyGrpc{
ClusterName: clusterName,
Authority: a.GrpcService.Authority,
},
},
Timeout: target.timeoutDurationPB(),
InitialMetadata: initialMetadata,
}, nil
}
// getClusterName returns the name of the cluster for the OpenTelemetry access logging service.
// If the extension is configured with an upstream OpenTelemetry access logging service then the name of the cluster for
// that upstream is returned. If the extension is configured with a URI, the only allowed host is `localhost`
// and the extension will insert a new cluster with the name "local_access_log", so we use that name.
func (a *AccessLog) getClusterName(cfg *cmn.RuntimeConfig, target *Target) (string, error) {
var err error
clusterName := LocalAccessLogClusterName
if target.isService() {
if clusterName, err = target.clusterName(cfg); err != nil {
return "", err
}
}
return clusterName, nil
}
// toEnvoyCluster returns an Envoy cluster for connecting to the OpenTelemetry access logging service.
// If the extension is configured with the OpenTelemetry access logging service locally via the URI set to localhost,
// this func will return a new cluster definition that will allow the proxy to connect to the OpenTelemetry access logging
// service running on localhost on the configured port.
//
// If the extension is configured with the OpenTelemetry access logging service as an upstream there is no need to insert
// a new cluster so this method returns nil.
func (a *AccessLog) toEnvoyCluster(_ *cmn.RuntimeConfig) (*envoy_cluster_v3.Cluster, error) {
target := a.GrpcService.Target
// If the target is an upstream we do not need to create a cluster. We will use the cluster of the upstream.
if target.isService() {
return nil, nil
}
host, port, err := target.addr()
if err != nil {
return nil, err
}
clusterType := &envoy_cluster_v3.Cluster_Type{Type: envoy_cluster_v3.Cluster_STATIC}
if host == localhost {
// If the host is "localhost" use a STRICT_DNS cluster type to perform DNS lookup.
clusterType = &envoy_cluster_v3.Cluster_Type{Type: envoy_cluster_v3.Cluster_STRICT_DNS}
}
var typedExtProtoOpts map[string]*anypb.Any
httpProtoOpts := &envoy_upstreams_http_v3.HttpProtocolOptions{
UpstreamProtocolOptions: &envoy_upstreams_http_v3.HttpProtocolOptions_ExplicitHttpConfig_{
ExplicitHttpConfig: &envoy_upstreams_http_v3.HttpProtocolOptions_ExplicitHttpConfig{
ProtocolConfig: &envoy_upstreams_http_v3.HttpProtocolOptions_ExplicitHttpConfig_Http2ProtocolOptions{},
},
},
}
httpProtoOptsAny, err := anypb.New(httpProtoOpts)
if err != nil {
return nil, err
}
typedExtProtoOpts = make(map[string]*anypb.Any)
typedExtProtoOpts["envoy.extensions.upstreams.http.v3.HttpProtocolOptions"] = httpProtoOptsAny
return &envoy_cluster_v3.Cluster{
Name: LocalAccessLogClusterName,
ClusterDiscoveryType: clusterType,
ConnectTimeout: target.timeoutDurationPB(),
LoadAssignment: &envoy_endpoint_v3.ClusterLoadAssignment{
ClusterName: LocalAccessLogClusterName,
Endpoints: []*envoy_endpoint_v3.LocalityLbEndpoints{
{
LbEndpoints: []*envoy_endpoint_v3.LbEndpoint{{
HostIdentifier: &envoy_endpoint_v3.LbEndpoint_Endpoint{
Endpoint: &envoy_endpoint_v3.Endpoint{
Address: &envoy_core_v3.Address{
Address: &envoy_core_v3.Address_SocketAddress{
SocketAddress: &envoy_core_v3.SocketAddress{
Address: host,
PortSpecifier: &envoy_core_v3.SocketAddress_PortValue{
PortValue: uint32(port),
},
},
},
},
},
},
}},
},
},
},
TypedExtensionProtocolOptions: typedExtProtoOpts,
}, nil
}
func (a *AccessLog) toEnvoyCommonGrpcAccessLogConfig(cfg *cmn.RuntimeConfig) (*envoy_extensions_access_loggers_grpc_v3.CommonGrpcAccessLogConfig, error) {
config := &envoy_extensions_access_loggers_grpc_v3.CommonGrpcAccessLogConfig{
LogName: a.LogName,
BufferSizeBytes: wrapperspb.UInt32(a.BufferSizeBytes),
FilterStateObjectsToLog: a.FilterStateObjectsToLog,
TransportApiVersion: envoy_core_v3.ApiVersion_V3,
}
if a.BufferFlushInterval != nil {
config.BufferFlushInterval = durationpb.New(*a.BufferFlushInterval)
}
if a.RetryPolicy != nil {
config.GrpcStreamRetryPolicy = a.RetryPolicy.toEnvoy()
}
grpcSvc, err := a.envoyGrpcService(cfg)
if err != nil {
return nil, err
}
config.GrpcService = grpcSvc
return config, nil
}
type GrpcService struct {
Target *Target
Authority string
InitialMetadata []*HeaderValue
}
func (v *GrpcService) normalize() {
if v == nil {
return
}
v.Target.normalize()
}
func (v *GrpcService) validate() error {
var resultErr error
if v == nil {
return resultErr
}
if v.Target == nil {
resultErr = multierror.Append(resultErr, fmt.Errorf("GrpcService.Target must be set"))
}
if err := v.Target.validate(); err != nil {
resultErr = multierror.Append(resultErr, err)
}
return resultErr
}
type HeaderValue struct {
Key string
Value string
}
func (h *HeaderValue) toEnvoy() *envoy_core_v3.HeaderValue {
if h == nil {
return nil
}
return &envoy_core_v3.HeaderValue{Key: h.Key, Value: h.Value}
}
type Target struct {
Service api.CompoundServiceName
URI string
Timeout string
timeout *time.Duration
host string
port int
}
// addr returns the host and port for the target when the target is a URI.
// It returns a non-nil error if the target is not a URI.
func (t Target) addr() (string, int, error) {
if !t.isURI() {
return "", 0, fmt.Errorf("target is not configured with a URI, set Target.URI")
}
return t.host, t.port, nil
}
// clusterName returns the cluster name for the target when the target is an upstream service.
// It searches through the upstreams in the provided runtime configuration and returns the name
// of the cluster for the first upstream service that matches the target service.
// It returns a non-nil error if a matching cluster is not found or if the target is not an
// upstream service.
func (t Target) clusterName(cfg *cmn.RuntimeConfig) (string, error) {
if !t.isService() {
return "", fmt.Errorf("target is not configured with an upstream service, set Target.Service")
}
for service, upstream := range cfg.Upstreams {
if service == t.Service {
for sni := range upstream.SNIs {
return sni, nil
}
}
}
return "", fmt.Errorf("no upstream definition found for service %q", t.Service.Name)
}
func (t Target) isService() bool {
return t.Service.Name != ""
}
func (t Target) isURI() bool {
return t.URI != ""
}
func (t *Target) normalize() {
if t == nil {
return
}
t.Service.Namespace = acl.NamespaceOrDefault(t.Service.Namespace)
t.Service.Partition = acl.PartitionOrDefault(t.Service.Partition)
}
// timeoutDurationPB returns the target's timeout as a *durationpb.Duration.
// It returns nil if the timeout has not been explicitly set.
func (t *Target) timeoutDurationPB() *durationpb.Duration {
if t == nil || t.timeout == nil {
return nil
}
return durationpb.New(*t.timeout)
}
func (t *Target) validate() error {
var err, resultErr error
if t == nil {
return resultErr
}
if t.isURI() == t.isService() {
resultErr = multierror.Append(resultErr, fmt.Errorf("exactly one of Target.Service or Target.URI must be set"))
}
if t.isURI() {
t.host, t.port, err = parseAddr(t.URI)
if err == nil {
switch t.host {
case localhost, localhostIPv4, localhostIPv6:
default:
resultErr = multierror.Append(resultErr,
fmt.Errorf("invalid host for Target.URI %q: expected %q, %q, or %q", t.URI, localhost, localhostIPv4, localhostIPv6))
}
} else {
resultErr = multierror.Append(resultErr, fmt.Errorf("invalid format for Target.URI %q: expected host:port", t.URI))
}
}
if t.Timeout != "" {
if d, err := time.ParseDuration(t.Timeout); err == nil {
t.timeout = &d
} else {
resultErr = multierror.Append(resultErr, fmt.Errorf("failed to parse Target.Timeout %q as a duration: %w", t.Timeout, err))
}
}
return resultErr
}
type RetryPolicy struct {
RetryBackOff *RetryBackOff
NumRetries uint32
}
func (r *RetryPolicy) normalize() {
if r == nil {
return
}
r.RetryBackOff.normalize()
}
func (r *RetryPolicy) toEnvoy() *envoy_core_v3.RetryPolicy {
if r == nil {
return nil
}
return &envoy_core_v3.RetryPolicy{
RetryBackOff: r.RetryBackOff.toEnvoy(),
NumRetries: wrapperspb.UInt32(r.NumRetries),
}
}
type RetryBackOff struct {
BaseInterval *time.Duration
MaxInterval *time.Duration
}
func (v *RetryBackOff) normalize() {
if v == nil {
return
}
if v.BaseInterval == nil {
v.BaseInterval = new(time.Duration)
*v.BaseInterval = time.Second
}
if v.MaxInterval == nil {
v.MaxInterval = new(time.Duration)
*v.MaxInterval = time.Second * 30
}
}
func (r *RetryBackOff) toEnvoy() *envoy_core_v3.BackoffStrategy {
if r == nil {
return nil
}
return &envoy_core_v3.BackoffStrategy{
BaseInterval: durationpb.New(*r.BaseInterval),
MaxInterval: durationpb.New(*r.MaxInterval),
}
}
func parseAddr(s string) (host string, port int, err error) {
// Strip the protocol if one was provided
if _, addr, hasProto := strings.Cut(s, "://"); hasProto {
s = addr
}
idx := strings.LastIndex(s, ":")
switch idx {
case -1, len(s) - 1:
err = fmt.Errorf("invalid input format %q: expected host:port", s)
case 0:
host = localhost
port, err = strconv.Atoi(s[idx+1:])
default:
host = s[:idx]
port, err = strconv.Atoi(s[idx+1:])
}
return
}

View File

@ -12,6 +12,7 @@ import (
awslambda "github.com/hashicorp/consul/agent/envoyextensions/builtin/aws-lambda"
extauthz "github.com/hashicorp/consul/agent/envoyextensions/builtin/ext-authz"
"github.com/hashicorp/consul/agent/envoyextensions/builtin/lua"
otelaccesslogging "github.com/hashicorp/consul/agent/envoyextensions/builtin/otel-access-logging"
propertyoverride "github.com/hashicorp/consul/agent/envoyextensions/builtin/property-override"
"github.com/hashicorp/consul/agent/envoyextensions/builtin/wasm"
"github.com/hashicorp/consul/api"
@ -21,11 +22,12 @@ import (
type extensionConstructor func(api.EnvoyExtension) (extensioncommon.EnvoyExtender, error)
var extensionConstructors = map[string]extensionConstructor{
api.BuiltinLuaExtension: lua.Constructor,
api.BuiltinAWSLambdaExtension: awslambda.Constructor,
api.BuiltinPropertyOverrideExtension: propertyoverride.Constructor,
api.BuiltinWasmExtension: wasm.Constructor,
api.BuiltinExtAuthzExtension: extauthz.Constructor,
api.BuiltinOTELAccessLoggingExtension: otelaccesslogging.Constructor,
api.BuiltinLuaExtension: lua.Constructor,
api.BuiltinAWSLambdaExtension: awslambda.Constructor,
api.BuiltinPropertyOverrideExtension: propertyoverride.Constructor,
api.BuiltinWasmExtension: wasm.Constructor,
api.BuiltinExtAuthzExtension: extauthz.Constructor,
}
// ConstructExtension attempts to lookup and build an extension from the registry with the

View File

@ -39,11 +39,12 @@ const (
)
const (
BuiltinAWSLambdaExtension string = "builtin/aws/lambda"
BuiltinExtAuthzExtension string = "builtin/ext-authz"
BuiltinLuaExtension string = "builtin/lua"
BuiltinPropertyOverrideExtension string = "builtin/property-override"
BuiltinWasmExtension string = "builtin/wasm"
BuiltinAWSLambdaExtension string = "builtin/aws/lambda"
BuiltinExtAuthzExtension string = "builtin/ext-authz"
BuiltinLuaExtension string = "builtin/lua"
BuiltinOTELAccessLoggingExtension string = "builtin/otel-access-logging"
BuiltinPropertyOverrideExtension string = "builtin/property-override"
BuiltinWasmExtension string = "builtin/wasm"
// BuiltinValidateExtension should not be exposed directly or accepted as a valid configured
// extension type, as it is only used indirectly via troubleshooting tools. It is included here
// for common reference alongside other builtin extensions.

View File

@ -21,6 +21,7 @@ Click on a property name to view additional details, including default values.
- [`Name`](#name): string | required | must be set to `builtin/ext-authz`
- [`Arguments`](#arguments): map | required
- [`ProxyType`](#arguments-proxytype): string | required | `connect-proxy`
- [`ListenerType`](#arguments-listenertype): string | required | `inbound`
- [`InsertOptions`](#arguments-insertoptions): map
- [`Location`](#arguments-insertoptions-location): string
- [FilterName](#arguments-insertoptions-filtername): string
@ -237,6 +238,18 @@ Specifies the type of Envoy proxy that this extension applies to. The extension
- This field is required.
- Data type: String
### `Arguments.ListenerType`
Specifies the type of listener the extension applies to. The listener type is either `inbound` or `outbound`. If the listener type is set to `inbound`, Consul applies the extension so the external authorization is enabled when other services in the mesh send messages to the service attached to the proxy. If the listener type is set to `outbound`, Consul applies the extension so the external authorization is enabled when the attached proxy sends messages to other services in the mesh.
#### Values
- Default: `inbound`
- This field is required.
- Data type is one of the following string values:
- `inbound`
- `outbound`
### `Arguments.InsertOptions`
Specifies options for defining the insertion point for the external authorization filter in the Envoy filter chain. By default, the external authorization filter is inserted as the first filter in the filter chain per the default setting for the [`Location`](#arguments-insertoptions-location) field.

View File

@ -0,0 +1,390 @@
---
layout: docs
page_title: OpenTelemetry Access Logging extension configuration reference
description: Learn how to configure the otel-access-logging Envoy extension, which is a builtin Consul extension that configures Envoy proxies to send access logs to OpenTelemetry collector service.
---
# OpenTelemetry Access Logging extension configuration reference
This topic describes how to configure the OpenTelemetry access logging Envoy extension, which configures Envoy proxies to send access logs to OpenTelemetry collector service. Refer to [Send access logs to OpenTelemetry collector service](/consul/docs/connect/proxies/envoy-extensions/usage/otel-access-logging) for usage information.
## Configuration model
The following list outlines the field hierarchy, data types, and requirements for the OpenTelemetry access logging configuration. Place the configuration inside the `EnvoyExtension.Arguments` field in the proxy defaults or service defaults configuration entry. Refer to the following documentation for additional information:
- [`EnvoyExtensions` in proxy defaults](/consul/docs/connect/config-entries/proxy-defaults#envoyextensions)
- [`EnvoyExtensions` in service defaults](/consul/docs/connect/config-entries/service-defaults#envoyextensions)
- [Envoy OpenTelemetry Access Logging Configuration documentation](https://www.envoyproxy.io/docs/envoy/latest/api-v3/extensions/access_loggers/open_telemetry/v3/logs_service.proto#extensions-access-loggers-open-telemetry-v3-opentelemetryaccesslogconfig)
Click on a property name to view additional details, including default values.
- [`Name`](#name): string | required | must be set to `builtin/otel-access-logging`
- [`Arguments`](#arguments): map | required
- [`ProxyType`](#arguments-proxytype): string | required | `connect-proxy`
- [`ListenerType`](#arguments-listenertype): string | required | `inbound`
- [`Config`](#arguments-config): map | required
- [`LogName`](#arguments-config-logname): string
- [`GrpcService`](#arguments-config-grpcservice): map
- [`Target`](#arguments-config-grpcservice-target): map | required
- [`Service`](#arguments-config-grpcservice-target-service): map
- [`Name`](#arguments-config-grpcservice-target-service): string
- [`Namespace`](#arguments-config-grpcservice-target-service): string | <EnterpriseAlert inline/>
- [`Partition`](#arguments-config-grpcservice-target-service): string | <EnterpriseAlert inline/>
- [`URI`](#arguments-config-grpcservice-target-uri): string
- [`Timeout`](#arguments-config-grpcservice-target-timeout): string | `1s`
- [`Authority`](#arguments-config-grpcservice-authority): string
- [`InitialMetadata`](#arguments-config-grpcservice-initialmetadata): list
- [`Key`](#arguments-config-grpcservice-initialmetadata): string
- [`Value`](#arguments-config-grpcservice-initialmetadata): string
- [`BufferFlushInterval`](#arguments-config-bufferflushinterval): string
- [`BufferSizeBytes`](#arguments-config-buffersizebytes): number
- [`FilterStateObjectsToLog`](#arguments-config-filterstateobjectstolog): list of strings
- [`RetryPolicy`](#arguments-config-retrypolicy): map
- [`RetryBackOff`](#arguments-config-retrypolicy-retrybackoff): map
- [`BaseInterval`](#arguments-config-retrypolicy-retrybackoff): string | `1s`
- [`MaxInterval`](#arguments-config-retrypolicy-retrybackoff): string | `30s`
- [`NumRetries`](#arguments-config-retrypolicy-numretries): number
- [`Body`](#arguments-config-body): string, number, boolean or list of bytes
- [`Attributes`](#arguments-config-attributes): map of string to string, number, boolean or list of bytes
- [`ResourceAttributes`](#arguments-config-resourceattributes): map of string to string, number, boolean or list of bytes
## Complete configuration
When each field is defined, an `otel-access-logging` configuration has the following form:
```hcl
Name = "builtin/otel-access-logging"
Arguments = {
ProxyType = "connect-proxy"
ListenerType = "<inbound or outbound>"
Config = {
LogName = "<user-readable name of the access log>"
GrpcService = {
Target = {
Service = {
Name = "<upstream service to send gRPC authorization requests to>"
Namespace = "<namespace containing the upstream service>"
Partition = "<partition containing the upstream service>"
}
URI = "<URI of the upstream service>"
Timeout = "1s"
}
Authority = "<authority header to send in the gRPC request>"
InitialMetadata = [
"<Key>" : "<value>"
]
}
BufferFlushInterval = "1s"
BufferSizeBytes = 16384
FilterStateObjectsToLog = [
"Additional filter state objects to log in filter_state_objects"
]
RetryPolicy = {
RetryBackOff = {
BaseInterval = "1s"
MaxInterval = "30s"
}
NumRetries = <uint32 value specifying the max number of retries>
}
Body = "Log Request Body"
Attributes = {
"<Key>" : "<value>"
}
ResourceAttributes = {
"<Key>" : "<value>"
}
```
## Specification
This section provides details about the fields you can configure for the OpenTelemetry Access Logging extension.
### `Name`
Specifies the name of the extension. Must be set to `builtin/otel-access-logging`.
#### Values
- Default: None
- This field is required.
- Data type: String value set to `builtin/otel-access-logging`.
### `Arguments`
Contains the global configuration for the extension.
#### Values
- Default: None
- This field is required.
- Data type: Map
### `Arguments.ProxyType`
Specifies the type of Envoy proxy that this extension applies to. The extension only applies to proxies that match this type and is ignored for all other proxy types. The only supported value is `connect-proxy`.
#### Values
- Default: `connect-proxy`
- This field is required.
- Data type: String
### `Arguments.ListenerType`
Specifies the type of listener the extension applies to. The listener type is either `inbound` or `outbound`. If the listener type is set to `inbound`, Consul applies the extension so the access logging is enabled when other services in the mesh send messages to the service attached to the proxy. If the listener type is set to `outbound`, Consul applies the extension so the access logging is enabled when the attached proxy sends messages to other services in the mesh.
#### Values
- Default: `inbound`
- This field is required.
- Data type is one of the following string values:
- `inbound`
- `outbound`
### `Arguments.Config`
Contains the configuration settings for the extension.
#### Values
- Default: None
- This field is required.
- Data type: Map
### `Arguments.Config.LogName`
Specifies the user-readable name of the access log to be returned in `StreamAccessLogsMessage.Identifier`. This allows the access log server to differentiate between different access logs coming from the same Envoy. If you leave it empty, it inherits the value from `ListenerType`.
#### Values
- Default: None
- Data type: String
### `Arguments.Config.GrpcService`
Specifies the OpenTelemetry Access Logging configuration for gRPC requests.
#### Values
- Default: None
- This field is required.
- Data type: Map
### `Arguments.Config.GrpcService.Target`
Configuration for specifying the service to send gRPC access logging requests to. The `Target` field may contain the following fields:
- [`Service`](#arguments-config-grpcservice-target-service) or [`Uri`](#arguments-config-grpcservice-target-uri)
- [`Timeout`](#arguments-config-grpcservice-target-timeout)
#### Values
- Default: None
- This field is required.
- Data type: Map
### `Arguments.Config.GrpcService.Target.Service`
Specifies the upstream OpenTelemetry collector service. Configure this field when access logging requests are sent to an upstream service within the service mesh. The service must be configured as an upstream of the service that the filter is applied to.
Configure either the `Service` field or the [`Uri`](#arguments-config-grpcservice-target-uri) field, but not both.
#### Values
- Default: None
- This field or [`Uri`](#arguments-config-grpcservice-target-uri) is required.
- Data type: Map
The following table describes how to configure parameters for the `Service` field:
| Parameter | Description | Data type | Default |
| ----------- | ---------------------------------------------------------------------------------------------------- | --------- | --------- |
| `Name` | Specifies the name of the upstream service. | String | None |
| `Namespace` | <EnterpriseAlert inline/> Specifies the Consul namespace that the upstream service belongs to. | String | `default` |
| `Partition` | <EnterpriseAlert inline/> Specifies the Consul admin partition that the upstream service belongs to. | String | `default` |
### `Arguments.Config.GrpcService.Target.Uri`
Specifies the URI of the OpenTelemetry collector service. Configure this field when you must provide an explicit URI to the OpenTelemetry collector service, such as cases in which the access logging service is running on the same host or pod. If set, the value of this field must be one of `localhost:<port>`, `127.0.0.1:<port>`, or `::1:<port>`.
Configure either the `Uri` field or the [`Service`](#arguments-config-grpcservice-target-service) field, but not both.
#### Values
- Default: None
- This field or [`Service`](#arguments-config-grpcservice-target-service) is required.
- Data type: String
### `Arguments.Config.GrpcService.Target.Timeout`
Specifies the maximum duration that a response can take to arrive upon request.
#### Values
- Default: `1s`
- Data type: String
### `Arguments.Config.GrpcService.Authority`
Specifies the authority header to send in the gRPC request. If this field is not set, the authority field is set to the cluster name. This field does not override the SNI that Envoy sends to the OpenTelemetry collector service.
#### Values
- Default: Cluster name
- Data type: String
### `Arguments.Config.GrpcService.InitialMetadata`
Specifies additional metadata to include in streams initiated to the `GrpcService`. You can specify metadata for injecting additional ad-hoc authorization headers, for example, `x-foo-bar: baz-key`. For more information, including details on header value syntax, refer to the [Envoy documentation on custom request headers](https://www.envoyproxy.io/docs/envoy/latest/configuration/http/http_conn_man/headers#config-http-conn-man-headers-custom-request-headers).
#### Values
- Default: None
- Data type: List of one or more key-value pairs:
- KEY: String
- VALUE: String
### `Arguments.Config.BufferFlushInterval`
Specifies an interval for flushing access logs to the gRPC stream. The logger flushes requests at the end of every interval or when the log reaches the batch size limit, whichever comes first.
#### Values
- Default: `1s`
- Data type: String
### `Arguments.Config.BufferSizeBytes`
Specifies the soft size limit in bytes for the access log entries buffer. The logger buffers requests until it reaches this limit or every time the flush interval elapses, whichever comes first. Set this field to `0` to disable batching.
#### Values
- Default: `16384`
- Data type: Integer
### `Arguments.Config.FilterStateObjectsToLog`
Specifies additional filter state objects to log in `filter_state_objects`. The logger calls `FilterState::Object::serializeAsProto` to serialize the filter state object.
#### Values
- Default: None
- Data type: List of String
### `Arguments.Config.RetryPolicy`
Defines a policy for retrying requests to the upstream service when fetching the plugin data. The `RetryPolicy` field is a map containing the following parameters:
- [`RetryBackoff`](#pluginconfig-vmconfig-code-remote-retrypolicy)
- [`NumRetries`](#pluginconfig-vmconfig-code-remote-numretries)
#### Values
- Default: None
- Data type: Map
### `Arguments.Config.RetryPolicy.RetryBackOff`
Specifies parameters that control retry backoff strategy.
#### Values
- Default: None
- Data type: Map
The following table describes the fields you can specify in the `RetryBackOff` map:
| Parameter | Description | Data type | Default |
| -------------- | ---------------------------------------------------------------------------------------------------------------------------------------------------------- | --------- | ------- |
| `BaseInterval` | Specifies the base interval for determining the next backoff computation. Set a value greater than `0` and less than or equal to the `MaxInterval` value. | String | `1s` |
| `MaxInterval` | Specifies the maximum interval between retries. Set the value greater than or equal to the `BaseInterval` value. | String | `10s` |
### `Arguments.Config.RetryPolicy.NumRetries`
Specifies the number of times Envoy retries to fetch plugin data if the initial attempt is unsuccessful.
#### Values
- Default: `1`
- Data type: Integer
### `Arguments.Config.Body`
Specifies OpenTelemetry [LogResource](https://github.com/open-telemetry/opentelemetry-proto/blob/main/opentelemetry/proto/logs/v1/logs.proto) fields, following [Envoy access logging formatting](https://www.envoyproxy.io/docs/envoy/latest/configuration/observability/access_log/usage). See body in the LogResource proto for more details.
#### Values
- Default: None
- Data type: String
### `Arguments.Config.Attributes`
Specifies `attributes` in the [LogResource](https://github.com/open-telemetry/opentelemetry-proto/blob/main/opentelemetry/proto/logs/v1/logs.proto). Refer to `attributes` in the LogResource proto for more details.
#### Values
- Default: None
- Data type: Map
### `Arguments.Config.ResourceAttributes`
Specifies OpenTelemetry [Resource](https://github.com/open-telemetry/opentelemetry-proto/blob/main/opentelemetry/proto/logs/v1/logs.proto#L51) attributes are filled with Envoy node information.
#### Values
- Default: None
- Data type: Map
## Examples
The following examples demonstrate common configuration patterns for specific use cases.
### OpenTelemetry Access Logging requests to URI
In the following example, a service defaults configuration entry contains an `otel-access-logging` configuration. The configuration allows the `api` service to make gRPC OpenTelemetry Access Logging requests to a service at `localhost:9191`:
```hcl
Kind = "service-defaults"
Name = "api"
EnvoyExtensions = [
{
Name = "builtin/otel-access-logging"
Arguments = {
ProxyType = "connect-proxy"
Config = {
GrpcService = {
Target = {
URI = "127.0.0.1:9191"
}
}
}
}
}
]
```
### Upstream OpenTelemetry Access Logging
In the following example, a service defaults configuration entry contains an `otel-access-logging` configuration. The configuration allows the `api` service to make gRPC OpenTelemetry Access Logging requests to a service named `otel-collector`:
```hcl
Kind = "service-defaults"
Name = "api"
EnvoyExtensions = [
{
Name = "builtin/otel-access-logging"
Arguments = {
ProxyType = "connect-proxy"
Config = {
GrpcService = {
Target = {
Service = {
Name = "otel-collector"
}
}
}
}
}
}
]
```

View File

@ -23,6 +23,7 @@ Envoy extensions enable additional service mesh functionality in Consul by chang
- External authorization
- Lua
- Lambda
- OpenTelemetry Access Logging
- Property override
- WebAssembly (Wasm)
@ -38,6 +39,10 @@ The `lambda` Envoy extension enables services to make requests to AWS Lambda fun
The `lua` Envoy extension enables HTTP Lua filters in your Consul Envoy proxies. It allows you to run Lua scripts during Envoy requests and responses from Consul-generated Envoy resources. Refer to the [Lua extension documentation](/consul/docs/connect/proxies/envoy-extensions/usage/lua) for more information.
### OpenTelemetry Access Logging
The `otel-access-logging` Envoy extension lets you configure Envoy proxies to send access logs to OpenTelemetry collector service. Refer to the [OpenTelemetry Access Logging extension documentation](/consul/docs/connect/proxies/envoy-extensions/usage/otel-access-logging) for more information.
### Property override
The `property-override` extension lets you set and unset individual properties on the Envoy resources that Consul generates. Use the extension instead of [escape-hatch overrides](/consul/docs/connect/proxies/envoy#escape-hatch-overrides) to enable advanced Envoy configuration. Refer to the [property override documentation](/consul/docs/connect/proxies/envoy-extensions/usage/property-override) for more information.

View File

@ -138,7 +138,6 @@ $ consul config write api-auth-service-defaults.hcl
```shell-session
$ consul config write api-auth-service-defaults.json
```
</Tab>

View File

@ -0,0 +1,146 @@
---
layout: docs
page_title: Send access logs to OpenTelemetry collector service
description: Learn how to use the `otel-access-logging` Envoy extension to send access logs to OpenTelemetry collector service.
---
# Send access logs to OpenTelemetry collector service
This topic describes how to use the OpenTelemetry Access Logging Envoy extension to send access logs to OpenTelemetry collector service.
## Workflow
Complete the following steps to use the OpenTelemetry Access Logging extension:
1. Configure an `EnvoyExtensions` block in a service defaults or proxy defaults configuration entry.
1. Apply the configuration entry.
## Add the `EnvoyExtensions`
Add Envoy extension configurations to a proxy defaults or service defaults configuration entry. Place the extension configuration in an `EnvoyExtensions` block in the configuration entry.
- When you configure Envoy extensions on proxy defaults, they apply to every service.
- When you configure Envoy extensions on service defaults, they apply to a specific service.
Consul applies Envoy extensions configured in proxy defaults before it applies extensions in service defaults. As a result, the Envoy extension configuration in service defaults may override configurations in proxy defaults.
The following example shows a service defaults configuration entry for the `api` service that directs the Envoy proxy to make gRPC OpenTelemetry Access Logging requests to the `otel-collector` service:
<Tabs>
<Tab heading="HCL" group="hcl">
<CodeBlockConfig filename="api-otel-collector-service-defaults.hcl">
```hcl
Kind = "service-defaults"
Name = "api"
EnvoyExtensions = [
{
Name = "builtin/otel-access-logging"
Arguments = {
ProxyType = "connect-proxy"
Config = {
GrpcService = {
Target = {
Service = {
Name = "otel-collector"
}
}
}
}
}
}
]
```
</CodeBlockConfig>
</Tab>
<Tab heading="JSON" group="json">
<CodeBlockConfig filename="api-otel-collector-service-defaults.json">
```json
"Kind": "service-defaults",
"Name": "api",
"EnvoyExtensions": [{
"Name": "builtin/otel-access-logging",
"Arguments": {
"ProxyType": "connect-proxy",
"Config": {
"GrpcService": {
"Target": {
"Service": {
"Name": "otel-collector"
}
}
}
}
}
}]
```
</CodeBlockConfig>
</Tab>
<Tab heading="Kubernetes" group="yaml">
<CodeBlockConfig filename="api-otel-collector-service-defaults.yaml">
```yaml
apiVersion: consul.hashicorp.com/v1alpha1
kind: ServiceDefaults
metadata:
name: api
namespace: default
spec:
envoyExtensions:
- name: builtin/otel-access-logging
arguments:
proxyType: connect-proxy
config:
grpcService:
target:
service:
name: otel-collector
namespace: otel-collector
```
</CodeBlockConfig>
</Tab>
</Tabs>
Refer to the [OpenTelemetry Access Logging extension configuration reference](/consul/docs/connect/proxies/envoy-extensions/configuration/otel-access-logging) for details on how to configure the extension.
Refer to the [proxy defaults configuration entry reference](/consul/docs/connect/config-entries/proxy-defaults) and [service defaults configuration entry reference](/consul/docs/connect/config-entries/service-defaults) for details on how to define the configuration entries.
!> **Warning:** Adding Envoy extensions default proxy configurations may have unintended consequences. We recommend configuring `EnvoyExtensions` in service defaults configuration entries in most cases.
### Unsupported Envoy configuration fields
The following Envoy configurations are not supported:
| Configuration | Workaround |
| ----------------------- | ------------------------------------------------------------------------------------------------------------------------------ |
| `transport_api_version` | Consul only supports v3 of the transport API. As a result, there is no workaround for implementing the behavior of this field. |
## Apply the configuration entry
If your network is deployed to virtual machines, use the `consul config write` command and specify the proxy defaults or service defaults configuration entry to apply the configuration. For Kubernetes-orchestrated networks, use the `kubectl apply` command. The following example applies the extension in a proxy defaults configuration entry.
<Tabs>
<Tab heading="HCL" group="hcl">
```shell-session
$ consul config write api-otel-collector-service-defaults.hcl
```
</Tab>
<Tab heading="JSON" group="json">
```shell-session
$ consul config write api-otel-collector-service-defaults.json
```
</Tab>
<Tab heading="Kubernetes" group="kubernetes">
```shell-session
$ kubectl apply -f api-otel-collector-service-defaults.yaml
```
</Tab>
</Tabs>

View File

@ -511,6 +511,10 @@
"title": "Invoke Lambda functions in Envoy proxies",
"path": "connect/proxies/envoy-extensions/usage/lambda"
},
{
"title": "Send access logs to OpenTelemetry collector service",
"path": "connect/proxies/envoy-extensions/usage/otel-access-logging"
},
{
"title": "Configure Envoy proxy properties",
"path": "connect/proxies/envoy-extensions/usage/property-override"
@ -528,6 +532,10 @@
"title": "External authorization",
"path": "connect/proxies/envoy-extensions/configuration/ext-authz"
},
{
"title": "OpenTelemetry Access Logging",
"path": "connect/proxies/envoy-extensions/configuration/otel-access-logging"
},
{
"title": "Property override",
"path": "connect/proxies/envoy-extensions/configuration/property-override"