From 0c80959b470205319226a60e3eef55fdbcc16073 Mon Sep 17 00:00:00 2001 From: Cole Laven <82364622+colelaven@users.noreply.github.com> Date: Thu, 19 Dec 2024 09:34:26 -0500 Subject: [PATCH] feat: Topology Processor (#1988) * fix: Shut down zombie goroutine in chronicleexporter (#2029) * Properly shut down chronicleexporter zombie goroutine * Fix lint * Fix the same problem for the GRPC workflow * initial structure & getting headers * progress * custom messages sending & receiving properly * more progress * more changes * cleanup * dont use TopologyInterval, TODO: remove TopologyInterval from BP Extension * cleanup * cleanup & tests * rm print statements * cleanup, tests, fix tests * fix bp extension logic, fix lint * add gatewayid parameter * fix concurrent map write * fix tests * fix test * cleanup names & data model, add ResourceNameHeader * fix resource name header * address pr feedback * fix lint * repo rename fixes * fix rebase issue * fix gomod versions * fix go mod * update topo proc go mod * fix flaky test * bump version to 1.68.0 --------- Co-authored-by: Ian Adams --- collector/collector.go | 4 +- extension/bindplaneextension/config.go | 2 +- extension/bindplaneextension/extension.go | 123 +- extension/bindplaneextension/go.mod | 3 + factories/processors.go | 2 + go.mod | 6 + internal/service/managed.go | 2 + internal/topology/bindplane_agent_topology.go | 18 + internal/topology/custom_message.go | 22 + internal/topology/go.mod | 3 + internal/topology/topology.go | 160 ++ opamp/observiq/observiq_client.go | 22 +- opamp/observiq/topology.go | 156 ++ processor/samplingprocessor/processor_test.go | 6 +- processor/topologyprocessor/README.md | 48 + .../topologyprocessor/bindplane_registry.go | 28 + processor/topologyprocessor/config.go | 68 + processor/topologyprocessor/config_test.go | 61 + processor/topologyprocessor/factory.go | 144 ++ processor/topologyprocessor/factory_test.go | 193 +++ processor/topologyprocessor/go.mod | 58 + processor/topologyprocessor/go.sum | 136 ++ processor/topologyprocessor/ocb_registry.go | 46 + processor/topologyprocessor/processor.go | 155 ++ processor/topologyprocessor/processor_test.go | 221 +++ .../testdata/logs/w3c-logs.yaml | 505 ++++++ .../testdata/metrics/host-metrics.yaml | 737 ++++++++ .../testdata/traces/bindplane-traces.yaml | 1530 +++++++++++++++++ 28 files changed, 4430 insertions(+), 29 deletions(-) create mode 100644 internal/topology/bindplane_agent_topology.go create mode 100644 internal/topology/custom_message.go create mode 100644 internal/topology/go.mod create mode 100644 internal/topology/topology.go create mode 100644 opamp/observiq/topology.go create mode 100644 processor/topologyprocessor/README.md create mode 100644 processor/topologyprocessor/bindplane_registry.go create mode 100644 processor/topologyprocessor/config.go create mode 100644 processor/topologyprocessor/config_test.go create mode 100644 processor/topologyprocessor/factory.go create mode 100644 processor/topologyprocessor/factory_test.go create mode 100644 processor/topologyprocessor/go.mod create mode 100644 processor/topologyprocessor/go.sum create mode 100644 processor/topologyprocessor/ocb_registry.go create mode 100644 processor/topologyprocessor/processor.go create mode 100644 processor/topologyprocessor/processor_test.go create mode 100644 processor/topologyprocessor/testdata/logs/w3c-logs.yaml create mode 100644 processor/topologyprocessor/testdata/metrics/host-metrics.yaml create mode 100644 processor/topologyprocessor/testdata/traces/bindplane-traces.yaml diff --git a/collector/collector.go b/collector/collector.go index 4a8b6f88a..1397d6610 100644 --- a/collector/collector.go +++ b/collector/collector.go @@ -25,6 +25,7 @@ import ( "github.com/observiq/bindplane-otel-collector/factories" "github.com/observiq/bindplane-otel-collector/internal/measurements" + "github.com/observiq/bindplane-otel-collector/internal/topology" "go.opentelemetry.io/collector/otelcol" "go.uber.org/zap" ) @@ -201,8 +202,9 @@ func (c *collector) Stop(ctx context.Context) { c.svc = nil - // After shutting down, we reset the throughputs measurements registry so it's fresh for the next collector startup. + // After shutting down, we reset the registries so they're fresh for the next collector startup. measurements.BindplaneAgentThroughputMeasurementsRegistry.Reset() + topology.BindplaneAgentTopologyRegistry.Reset() } // Restart will restart the collector. It will also reset the status channel. diff --git a/extension/bindplaneextension/config.go b/extension/bindplaneextension/config.go index c90135eea..ddea5c574 100644 --- a/extension/bindplaneextension/config.go +++ b/extension/bindplaneextension/config.go @@ -26,7 +26,7 @@ type Config struct { // Labels in "k1=v1,k2=v2" format Labels string `mapstructure:"labels"` // Component ID of the opamp extension. If not specified, then - // this extension will not generate any custom messages for throughput metrics. + // this extension will not generate any custom messages for throughput metrics or topology. OpAMP component.ID `mapstructure:"opamp"` // MeasurementsInterval is the interval on which to report measurements. // Measurements reporting is disabled if this duration is 0. diff --git a/extension/bindplaneextension/extension.go b/extension/bindplaneextension/extension.go index 664189f5b..d728e5948 100644 --- a/extension/bindplaneextension/extension.go +++ b/extension/bindplaneextension/extension.go @@ -16,6 +16,7 @@ package bindplaneextension import ( "context" + "encoding/json" "errors" "fmt" "sync" @@ -23,6 +24,7 @@ import ( "github.com/golang/snappy" "github.com/observiq/bindplane-otel-collector/internal/measurements" + "github.com/observiq/bindplane-otel-collector/internal/topology" "github.com/open-telemetry/opamp-go/client/types" "github.com/open-telemetry/opentelemetry-collector-contrib/extension/opampcustommessages" "go.opentelemetry.io/collector/component" @@ -31,43 +33,56 @@ import ( ) type bindplaneExtension struct { - logger *zap.Logger - cfg *Config - ctmr *measurements.ResettableThroughputMeasurementsRegistry - customCapabilityHandler opampcustommessages.CustomCapabilityHandler - doneChan chan struct{} - wg *sync.WaitGroup + logger *zap.Logger + cfg *Config + measurementsRegistry *measurements.ResettableThroughputMeasurementsRegistry + topologyRegistry *topology.ResettableTopologyRegistry + customCapabilityHandlerThroughput opampcustommessages.CustomCapabilityHandler + customCapabilityHandlerTopology opampcustommessages.CustomCapabilityHandler + + doneChan chan struct{} + wg *sync.WaitGroup } func newBindplaneExtension(logger *zap.Logger, cfg *Config) *bindplaneExtension { return &bindplaneExtension{ - logger: logger, - cfg: cfg, - ctmr: measurements.NewResettableThroughputMeasurementsRegistry(false), - doneChan: make(chan struct{}), - wg: &sync.WaitGroup{}, + logger: logger, + cfg: cfg, + measurementsRegistry: measurements.NewResettableThroughputMeasurementsRegistry(false), + topologyRegistry: topology.NewResettableTopologyRegistry(), + doneChan: make(chan struct{}), + wg: &sync.WaitGroup{}, } } func (b *bindplaneExtension) Start(_ context.Context, host component.Host) error { var emptyComponentID component.ID - // Set up measurements if enabled - if b.cfg.OpAMP != emptyComponentID && b.cfg.MeasurementsInterval > 0 { + // Set up custom capabilities if enabled + if b.cfg.OpAMP != emptyComponentID { err := b.setupCustomCapabilities(host) if err != nil { return fmt.Errorf("setup capability handler: %w", err) } + if b.cfg.MeasurementsInterval > 0 { + b.wg.Add(1) + go b.reportMetricsLoop() + } + b.wg.Add(1) - go b.reportMetricsLoop() + go b.reportTopologyLoop() } return nil } func (b *bindplaneExtension) RegisterThroughputMeasurements(processorID string, measurements *measurements.ThroughputMeasurements) error { - return b.ctmr.RegisterThroughputMeasurements(processorID, measurements) + return b.measurementsRegistry.RegisterThroughputMeasurements(processorID, measurements) +} + +func (b *bindplaneExtension) RegisterTopologyState(processorID string, topology *topology.TopoState) error { + return b.topologyRegistry.RegisterTopologyState(processorID, topology) } func (b *bindplaneExtension) setupCustomCapabilities(host component.Host) error { @@ -82,9 +97,16 @@ func (b *bindplaneExtension) setupCustomCapabilities(host component.Host) error } var err error - b.customCapabilityHandler, err = registry.Register(measurements.ReportMeasurementsV1Capability) + if b.cfg.MeasurementsInterval > 0 { + b.customCapabilityHandlerThroughput, err = registry.Register(measurements.ReportMeasurementsV1Capability) + if err != nil { + return fmt.Errorf("register custom measurements capability: %w", err) + } + } + + b.customCapabilityHandlerTopology, err = registry.Register(topology.ReportTopologyCapability) if err != nil { - return fmt.Errorf("register custom capability: %w", err) + return fmt.Errorf("register custom topology capability: %w", err) } return nil @@ -119,7 +141,7 @@ func (b *bindplaneExtension) reportMetricsLoop() { } func (b *bindplaneExtension) reportMetrics() error { - m := b.ctmr.OTLPMeasurements(b.cfg.ExtraMeasurementsAttributes) + m := b.measurementsRegistry.OTLPMeasurements(b.cfg.ExtraMeasurementsAttributes) // Send metrics as snappy-encoded otlp proto marshaller := pmetric.ProtoMarshaler{} @@ -130,7 +152,7 @@ func (b *bindplaneExtension) reportMetrics() error { encoded := snappy.Encode(nil, marshalled) for { - sendingChannel, err := b.customCapabilityHandler.SendMessage(measurements.ReportMeasurementsType, encoded) + sendingChannel, err := b.customCapabilityHandlerThroughput.SendMessage(measurements.ReportMeasurementsType, encoded) switch { case err == nil: return nil @@ -138,7 +160,60 @@ func (b *bindplaneExtension) reportMetrics() error { <-sendingChannel continue default: - return fmt.Errorf("send custom message: %w", err) + return fmt.Errorf("send custom throughput message: %w", err) + } + } +} + +func (b *bindplaneExtension) reportTopologyLoop() { + defer b.wg.Done() + + var topologyInterval time.Duration + select { + case <-b.doneChan: + return + case topologyInterval = <-b.topologyRegistry.SetIntervalChan(): + if topologyInterval <= 0 { + return + } + } + + t := time.NewTicker(topologyInterval) + defer t.Stop() + + for { + select { + case <-t.C: + err := b.reportTopology() + if err != nil { + b.logger.Error("Failed to report topology.", zap.Error(err)) + } + case <-b.doneChan: + return + } + } +} + +func (b *bindplaneExtension) reportTopology() error { + ts := b.topologyRegistry.TopologyInfos() + + // Send topology state snappy-encoded + marshalled, err := json.Marshal(ts) + if err != nil { + return fmt.Errorf("marshal topology state: %w", err) + } + + encoded := snappy.Encode(nil, marshalled) + for { + sendingChannel, err := b.customCapabilityHandlerTopology.SendMessage(topology.ReportTopologyType, encoded) + switch { + case err == nil: + return nil + case errors.Is(err, types.ErrCustomMessagePending): + <-sendingChannel + continue + default: + return fmt.Errorf("send custom topology message: %w", err) } } } @@ -158,8 +233,12 @@ func (b *bindplaneExtension) Shutdown(ctx context.Context) error { case <-waitgroupDone: // OK } - if b.customCapabilityHandler != nil { - b.customCapabilityHandler.Unregister() + if b.customCapabilityHandlerThroughput != nil { + b.customCapabilityHandlerThroughput.Unregister() + } + + if b.customCapabilityHandlerTopology != nil { + b.customCapabilityHandlerTopology.Unregister() } return nil diff --git a/extension/bindplaneextension/go.mod b/extension/bindplaneextension/go.mod index 044d67dcc..0942b50a9 100644 --- a/extension/bindplaneextension/go.mod +++ b/extension/bindplaneextension/go.mod @@ -5,6 +5,7 @@ go 1.22.7 require ( github.com/golang/snappy v0.0.4 github.com/observiq/bindplane-otel-collector/internal/measurements v1.68.0 + github.com/observiq/bindplane-otel-collector/internal/topology v1.68.0 github.com/open-telemetry/opamp-go v0.17.0 github.com/open-telemetry/opentelemetry-collector-contrib/extension/opampcustommessages v0.116.0 github.com/stretchr/testify v1.10.0 @@ -51,3 +52,5 @@ require ( ) replace github.com/observiq/bindplane-otel-collector/internal/measurements => ../../internal/measurements + +replace github.com/observiq/bindplane-otel-collector/internal/topology => ../../internal/topology diff --git a/factories/processors.go b/factories/processors.go index 1dd2197b5..3801ba1b3 100644 --- a/factories/processors.go +++ b/factories/processors.go @@ -27,6 +27,7 @@ import ( "github.com/observiq/bindplane-otel-collector/processor/samplingprocessor" "github.com/observiq/bindplane-otel-collector/processor/spancountprocessor" "github.com/observiq/bindplane-otel-collector/processor/throughputmeasurementprocessor" + "github.com/observiq/bindplane-otel-collector/processor/topologyprocessor" "github.com/observiq/bindplane-otel-collector/processor/unrollprocessor" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/attributesprocessor" "github.com/open-telemetry/opentelemetry-collector-contrib/processor/cumulativetodeltaprocessor" @@ -87,6 +88,7 @@ var defaultProcessors = []processor.Factory{ spanprocessor.NewFactory(), throughputmeasurementprocessor.NewFactory(), tailsamplingprocessor.NewFactory(), + topologyprocessor.NewFactory(), transformprocessor.NewFactory(), unrollprocessor.NewFactory(), } diff --git a/go.mod b/go.mod index bb291d034..7371dfb45 100644 --- a/go.mod +++ b/go.mod @@ -199,6 +199,8 @@ require ( ) require ( + github.com/observiq/bindplane-otel-collector/internal/topology v1.68.0 + github.com/observiq/bindplane-otel-collector/processor/topologyprocessor v1.68.0 github.com/open-telemetry/opentelemetry-collector-contrib/confmap/provider/aesprovider v0.116.0 github.com/open-telemetry/opentelemetry-collector-contrib/processor/intervalprocessor v0.116.0 go.opentelemetry.io/collector/extension/extensiontest v0.116.0 @@ -853,6 +855,8 @@ replace github.com/observiq/bindplane-otel-collector/processor/lookupprocessor = replace github.com/observiq/bindplane-otel-collector/processor/unrollprocessor => ./processor/unrollprocessor +replace github.com/observiq/bindplane-otel-collector/processor/topologyprocessor => ./processor/topologyprocessor + replace github.com/observiq/bindplane-otel-collector/expr => ./expr replace github.com/observiq/bindplane-otel-collector/counter => ./counter @@ -875,6 +879,8 @@ replace github.com/observiq/bindplane-otel-collector/internal/report => ./intern replace github.com/observiq/bindplane-otel-collector/internal/measurements => ./internal/measurements +replace github.com/observiq/bindplane-otel-collector/internal/topology => ./internal/topology + replace github.com/observiq/bindplane-otel-collector/receiver/splunksearchapireceiver => ./receiver/splunksearchapireceiver // Does not build with windows and only used in configschema executable diff --git a/internal/service/managed.go b/internal/service/managed.go index c21b8634b..d0b27b5a4 100644 --- a/internal/service/managed.go +++ b/internal/service/managed.go @@ -20,6 +20,7 @@ import ( "fmt" "github.com/observiq/bindplane-otel-collector/internal/measurements" + "github.com/observiq/bindplane-otel-collector/internal/topology" "github.com/observiq/bindplane-otel-collector/collector" "github.com/observiq/bindplane-otel-collector/internal/version" @@ -57,6 +58,7 @@ func NewManagedCollectorService(col collector.Collector, logger *zap.Logger, man CollectorConfigPath: collectorConfigPath, LoggerConfigPath: loggerConfigPath, MeasurementsReporter: measurements.BindplaneAgentThroughputMeasurementsRegistry, + TopologyReporter: topology.BindplaneAgentTopologyRegistry, } // Create new client diff --git a/internal/topology/bindplane_agent_topology.go b/internal/topology/bindplane_agent_topology.go new file mode 100644 index 000000000..6089c187f --- /dev/null +++ b/internal/topology/bindplane_agent_topology.go @@ -0,0 +1,18 @@ +// Copyright observIQ, 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. + +package topology + +// BindplaneAgentTopologyRegistry is the registry singleton used by bindplane agent to track topology state +var BindplaneAgentTopologyRegistry = NewResettableTopologyRegistry() diff --git a/internal/topology/custom_message.go b/internal/topology/custom_message.go new file mode 100644 index 000000000..99129eb42 --- /dev/null +++ b/internal/topology/custom_message.go @@ -0,0 +1,22 @@ +// Copyright observIQ, 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. + +package topology + +const ( + // ReportTopologyCapability is the capability needed to report topology to bindplane + ReportTopologyCapability = "com.bindplane.topology" + // ReportTopologyType is the type for reporting topology to BindPlane + ReportTopologyType = "reportTopology" +) diff --git a/internal/topology/go.mod b/internal/topology/go.mod new file mode 100644 index 000000000..d18622410 --- /dev/null +++ b/internal/topology/go.mod @@ -0,0 +1,3 @@ +module github.com/observiq/bindplane-otel-collector/internal/topology + +go 1.22.6 diff --git a/internal/topology/topology.go b/internal/topology/topology.go new file mode 100644 index 000000000..b963dddf5 --- /dev/null +++ b/internal/topology/topology.go @@ -0,0 +1,160 @@ +// Copyright observIQ, 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. + +// Package topology provides code to help manage topology updates for BindPlane and the topology processor. +package topology + +import ( + "context" + "fmt" + "sync" + "time" +) + +// TopoRegistry represents a registry for the topology processor to register their TopologyState. +type TopoRegistry interface { + // RegisterTopologyState registers the topology state for the given processor. + // It should return an error if the processor has already been registered. + RegisterTopologyState(processorID string, data *TopoState) error + SetIntervalChan() chan time.Duration + Reset() +} + +// TopoState represents the data captured through topology processors. +type TopoState struct { + Topology topology + mux sync.Mutex +} + +type topology struct { + // GatewaySource is the gateway source that the entries in the route table point to + GatewaySource GatewayInfo + // RouteTable is a map of gateway destinations to the time at which they were last detected + RouteTable map[GatewayInfo]time.Time +} + +// GatewayInfo represents a bindplane gateway source or destination +type GatewayInfo struct { + // OrganizationID is the organizationID where this gateway dest/source lives + OrganizationID string `json:"organizationID"` + // AccountID is the accountID where this gateway dest/source lives + AccountID string `json:"accountID"` + // Configuration is the name of the configuration where this gateway dest/source lives + Configuration string `json:"configuration"` + // GatewayID is the ComponentID of a gateway source, or the resource name of a gateway destination + GatewayID string `json:"gatewayID"` +} + +// GatewayRecord represents a gateway destination and the time it was last detected +type GatewayRecord struct { + // Gateway represents a gateway destinations + Gateway GatewayInfo `json:"gateway"` + // LastUpdated is a timestamp of the last time a message w/ topology headers was detected from the above gateway destination + LastUpdated time.Time `json:"lastUpdated"` +} + +// TopoInfo represents a gateway source & the gateway destinations that point to it. +type TopoInfo struct { + GatewaySource GatewayInfo `json:"gatewaySource"` + GatewayDestinations []GatewayRecord `json:"gatewayDestinations"` +} + +// NewTopologyState initializes a new TopologyState +func NewTopologyState(gw GatewayInfo) (*TopoState, error) { + return &TopoState{ + Topology: topology{ + GatewaySource: gw, + RouteTable: make(map[GatewayInfo]time.Time), + }, + mux: sync.Mutex{}, + }, nil +} + +// UpsertRoute upserts given route. +func (ts *TopoState) UpsertRoute(_ context.Context, gw GatewayInfo) { + ts.mux.Lock() + defer ts.mux.Unlock() + + ts.Topology.RouteTable[gw] = time.Now() +} + +// ResettableTopologyRegistry is a concrete version of TopologyDataRegistry that is able to be reset. +type ResettableTopologyRegistry struct { + topology *sync.Map + setIntervalChan chan time.Duration +} + +// NewResettableTopologyRegistry creates a new ResettableTopologyRegistry +func NewResettableTopologyRegistry() *ResettableTopologyRegistry { + return &ResettableTopologyRegistry{ + topology: &sync.Map{}, + setIntervalChan: make(chan time.Duration, 1), + } +} + +// RegisterTopologyState registers the TopologyState with the registry. +func (rtsr *ResettableTopologyRegistry) RegisterTopologyState(processorID string, topoState *TopoState) error { + _, alreadyExists := rtsr.topology.LoadOrStore(processorID, topoState) + if alreadyExists { + return fmt.Errorf("topology for processor %q was already registered", processorID) + } + + return nil +} + +// Reset unregisters all topology states in this registry +func (rtsr *ResettableTopologyRegistry) Reset() { + rtsr.topology = &sync.Map{} +} + +// SetIntervalChan returns the setIntervalChan +func (rtsr *ResettableTopologyRegistry) SetIntervalChan() chan time.Duration { + return rtsr.setIntervalChan +} + +// TopologyInfos returns all the topology data in this registry. +func (rtsr *ResettableTopologyRegistry) TopologyInfos() []TopoInfo { + states := []topology{} + + rtsr.topology.Range(func(_, value any) bool { + ts := value.(*TopoState) + states = append(states, ts.Topology) + return true + }) + + ti := []TopoInfo{} + for _, ts := range states { + curInfo := TopoInfo{} + curInfo.GatewaySource.OrganizationID = ts.GatewaySource.OrganizationID + curInfo.GatewaySource.AccountID = ts.GatewaySource.AccountID + curInfo.GatewaySource.Configuration = ts.GatewaySource.Configuration + curInfo.GatewaySource.GatewayID = ts.GatewaySource.GatewayID + for gw, updated := range ts.RouteTable { + curInfo.GatewayDestinations = append(curInfo.GatewayDestinations, GatewayRecord{ + Gateway: GatewayInfo{ + OrganizationID: gw.OrganizationID, + AccountID: gw.AccountID, + Configuration: gw.Configuration, + GatewayID: gw.GatewayID, + }, + LastUpdated: updated.UTC(), + }) + } + if len(curInfo.GatewayDestinations) > 0 { + ti = append(ti, curInfo) + } + } + + return ti +} diff --git a/opamp/observiq/observiq_client.go b/opamp/observiq/observiq_client.go index 67094b075..8973be009 100644 --- a/opamp/observiq/observiq_client.go +++ b/opamp/observiq/observiq_client.go @@ -28,6 +28,7 @@ import ( "github.com/observiq/bindplane-otel-collector/collector" "github.com/observiq/bindplane-otel-collector/internal/measurements" "github.com/observiq/bindplane-otel-collector/internal/report" + "github.com/observiq/bindplane-otel-collector/internal/topology" "github.com/observiq/bindplane-otel-collector/internal/version" "github.com/observiq/bindplane-otel-collector/opamp" "github.com/observiq/bindplane-otel-collector/packagestate" @@ -66,6 +67,7 @@ type Client struct { updatingPackage bool reportManager *report.Manager measurementsSender *measurementsSender + topologySender *topologySender // To signal if we are disconnecting already and not take any actions on connection failures disconnecting bool @@ -90,6 +92,7 @@ type NewClientArgs struct { CollectorConfigPath string LoggerConfigPath string MeasurementsReporter MeasurementsReporter + TopologyReporter TopologyReporter } // NewClient creates a new OpAmp client @@ -149,6 +152,7 @@ func NewClient(args *NewClientArgs) (opamp.Client, error) { err = observiqClient.opampClient.SetCustomCapabilities(&protobufs.CustomCapabilities{ Capabilities: []string{ measurements.ReportMeasurementsV1Capability, + topology.ReportTopologyCapability, }, }) if err != nil { @@ -164,6 +168,13 @@ func NewClient(args *NewClientArgs) (opamp.Client, error) { args.Config.ExtraMeasurementsAttributes, ) + // Create topology sender + observiqClient.topologySender = newTopologySender( + clientLogger, + args.TopologyReporter, + observiqClient.opampClient, + ) + return observiqClient, nil } @@ -344,12 +355,19 @@ func (c *Client) onMessageFuncHandler(ctx context.Context, msg *types.MessageDat } if msg.CustomCapabilities != nil { if slices.Contains(msg.CustomCapabilities.Capabilities, measurements.ReportMeasurementsV1Capability) { - c.logger.Info("Server supports custom message measurements, starting sender.") + c.logger.Info("Server supports custom throughput message measurements, starting measurements sender.") c.measurementsSender.Start() } else { - c.logger.Info("Server does not support custom message measurements, stopping sender.") + c.logger.Info("Server does not support custom throughput message measurements, stopping measurements sender.") c.measurementsSender.Stop() } + if slices.Contains(msg.CustomCapabilities.Capabilities, topology.ReportTopologyCapability) { + c.logger.Info("Server supports custom topology messages, starting topology sender.") + c.topologySender.Start() + } else { + c.logger.Info("Server does not support custom topology messages, stopping topology sender.") + c.topologySender.Stop() + } } } diff --git a/opamp/observiq/topology.go b/opamp/observiq/topology.go new file mode 100644 index 000000000..96944a7be --- /dev/null +++ b/opamp/observiq/topology.go @@ -0,0 +1,156 @@ +// Copyright observIQ, 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. + +package observiq + +import ( + "encoding/json" + "errors" + "sync" + "time" + + "github.com/golang/snappy" + "github.com/observiq/bindplane-otel-collector/internal/topology" + "github.com/open-telemetry/opamp-go/client" + "github.com/open-telemetry/opamp-go/client/types" + "github.com/open-telemetry/opamp-go/protobufs" + "go.uber.org/zap" +) + +// TopologyReporter represents an object that reports topology state. +type TopologyReporter interface { + TopologyInfos() []topology.TopoInfo + SetIntervalChan() chan time.Duration +} + +// topologySender is a struct that handles periodically sending topology state via a custom message to an OpAMP endpoint. +type topologySender struct { + logger *zap.Logger + reporter TopologyReporter + opampClient client.OpAMPClient + + mux *sync.Mutex + isRunning bool + done chan struct{} + wg *sync.WaitGroup +} + +func newTopologySender(l *zap.Logger, reporter TopologyReporter, opampClient client.OpAMPClient) *topologySender { + return &topologySender{ + logger: l, + reporter: reporter, + opampClient: opampClient, + + mux: &sync.Mutex{}, + isRunning: false, + done: make(chan struct{}), + wg: &sync.WaitGroup{}, + } +} + +// Start starts the sender. It may be called multiple times, even if the sender is already started. +func (ts *topologySender) Start() { + ts.mux.Lock() + defer ts.mux.Unlock() + + if ts.isRunning { + return + } + + ts.isRunning = true + + ts.wg.Add(1) + go func() { + defer ts.wg.Done() + ts.loop() + }() +} + +func (ts *topologySender) Stop() { + ts.mux.Lock() + defer ts.mux.Unlock() + + if !ts.isRunning { + return + } + + close(ts.done) + ts.wg.Wait() + + ts.isRunning = false +} + +func (ts *topologySender) loop() { + t := newTicker() + defer t.Stop() + + for { + select { + case setInterval := <-ts.reporter.SetIntervalChan(): + t.SetInterval(setInterval) + case <-ts.done: + return + case <-t.Chan(): + if ts.reporter == nil { + // Continue if no reporter available + ts.logger.Debug("No reporter, skipping sending topology.") + continue + } + + topoState := ts.reporter.TopologyInfos() + if len(topoState) == 0 { + // don't report empty payloads + continue + } + + // Send topology state snappy-encoded + marshalled, err := json.Marshal(topoState) + if err != nil { + ts.logger.Error("Failed to marshal topology state.", zap.Error(err)) + continue + } + + encoded := snappy.Encode(nil, marshalled) + + cm := &protobufs.CustomMessage{ + Capability: topology.ReportTopologyCapability, + Type: topology.ReportTopologyType, + Data: encoded, + } + + for i := 0; i < maxSendRetries; i++ { + sendingChannel, err := ts.opampClient.SendCustomMessage(cm) + switch { + case err == nil: // OK + case errors.Is(err, types.ErrCustomMessagePending): + if i == maxSendRetries-1 { + // Bail out early, since we aren't going to try to send again + ts.logger.Warn("Topology were blocked by other custom messages, skipping...", zap.Int("retries", maxSendRetries)) + break + } + + select { + case <-sendingChannel: + continue + case <-ts.done: + return + } + default: + ts.logger.Error("Failed to report topology", zap.Error(err)) + } + break + } + } + } +} diff --git a/processor/samplingprocessor/processor_test.go b/processor/samplingprocessor/processor_test.go index ef759f43a..e68f6d387 100644 --- a/processor/samplingprocessor/processor_test.go +++ b/processor/samplingprocessor/processor_test.go @@ -274,7 +274,7 @@ func Test_completeResourceDropping(t *testing.T) { ld := plog.NewLogs() for i := 0; i < 2; i++ { rl := ld.ResourceLogs().AppendEmpty() - for j := 0; j < 3; j++ { + for j := 0; j < 10; j++ { sl := rl.ScopeLogs().AppendEmpty() lr := sl.LogRecords().AppendEmpty() @@ -308,7 +308,7 @@ func Test_completeResourceDropping(t *testing.T) { td := ptrace.NewTraces() for i := 0; i < 2; i++ { rt := td.ResourceSpans().AppendEmpty() - for j := 0; j < 3; j++ { + for j := 0; j < 10; j++ { st := rt.ScopeSpans().AppendEmpty() sd := st.Spans().AppendEmpty() @@ -342,7 +342,7 @@ func Test_completeResourceDropping(t *testing.T) { md := pmetric.NewMetrics() for i := 0; i < 2; i++ { rm := md.ResourceMetrics().AppendEmpty() - for j := 0; j < 3; j++ { + for j := 0; j < 10; j++ { sm := rm.ScopeMetrics().AppendEmpty() m := sm.Metrics().AppendEmpty() diff --git a/processor/topologyprocessor/README.md b/processor/topologyprocessor/README.md new file mode 100644 index 000000000..81703e1bc --- /dev/null +++ b/processor/topologyprocessor/README.md @@ -0,0 +1,48 @@ +# Topology Processor +This processor utilizes request headers to provide extended topology functionality in BindPlane. + +## Minimum agent versions +- Introduced: [v1.68.0](https://github.com/observIQ/bindplane-otel-collector/releases/tag/v1.68.0) + +## Supported pipelines: +- Logs +- Metrics +- Traces + +## Configuration +| Field | Type | Default | Description | +|----------------------|-----------|---------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `interval` | duration | `1m` | The interval at which topology data is sent to Bindplane via OpAMP. | +| `organizationID` | string | | The Organization ID of the Bindplane configuration where this processor is running. | +| `accountID` | string | | The Account ID of the Bindplane configuration where this processor is running. | +| `configuration` | string | | The name of the Bindplane configuration this processor is running on. | + + +### Example configuration + +```yaml +receivers: + filelog: + inclucde: ["/var/log/*.log"] + +processors: + topology: + interval: 1m + organizationID: "myOrganizationID" + accountID: "myAccountID" + configuration: "myConfiguration" + + +exporters: + googlecloud: + +service: + pipelines: + logs: + receivers: + - filelog + processors: + - topology + exporters: + - googlecloud +``` diff --git a/processor/topologyprocessor/bindplane_registry.go b/processor/topologyprocessor/bindplane_registry.go new file mode 100644 index 000000000..45cbe04de --- /dev/null +++ b/processor/topologyprocessor/bindplane_registry.go @@ -0,0 +1,28 @@ +// // Copyright observIQ, 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. + +//go:build bindplane + +package topologyprocessor + +import ( + "github.com/observiq/bindplane-otel-collector/internal/topology" + "go.opentelemetry.io/collector/component" +) + +// GetTopologyRegistry returns the topology registry that should be registered to based on the component ID. +// nil, nil may be returned by this function. In this case, the processor should not register it's topology state anywhere. +func GetTopologyRegistry(host component.Host, bindplane component.ID) (topology.TopoRegistry, error) { + return topology.BindplaneAgentTopologyRegistry, nil +} diff --git a/processor/topologyprocessor/config.go b/processor/topologyprocessor/config.go new file mode 100644 index 000000000..d5763742b --- /dev/null +++ b/processor/topologyprocessor/config.go @@ -0,0 +1,68 @@ +// Copyright observIQ, 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. + +// Package topologyprocessor collects metrics, traces, and logs for +package topologyprocessor + +import ( + "errors" + "time" + + "go.opentelemetry.io/collector/component" +) + +const defaultInterval = time.Minute + +// Config is the configuration for the processor +type Config struct { + // Enabled controls whether this processor is enabled or not. + Enabled bool `mapstructure:"enabled"` + + // Interval is the interval at which this processor sends topology messages to BindPlane + Interval time.Duration `mapstructure:"interval"` + + // Bindplane extension to use in order to report topology. Optional. + BindplaneExtension component.ID `mapstructure:"bindplane_extension"` + + // Name of the Config where this processor is present + Configuration string `mapstructure:"configuration"` + + // OrganizationID of the Org where this processor is present + OrganizationID string `mapstructure:"organizationID"` + + // AccountID of the Account where this processor is present + AccountID string `mapstructure:"accountID"` +} + +// Validate validates the processor configuration +func (cfg Config) Validate() error { + // Processor not enabled no validation needed + if !cfg.Enabled { + return nil + } + + if cfg.Configuration == "" { + return errors.New("`configuration` must be specified") + } + + if cfg.OrganizationID == "" { + return errors.New("`organizationID` must be specified") + } + + if cfg.AccountID == "" { + return errors.New("`accountID` must be specified") + } + + return nil +} diff --git a/processor/topologyprocessor/config_test.go b/processor/topologyprocessor/config_test.go new file mode 100644 index 000000000..a7d43a203 --- /dev/null +++ b/processor/topologyprocessor/config_test.go @@ -0,0 +1,61 @@ +// Copyright observIQ, 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. + +package topologyprocessor + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestConfigValidate(t *testing.T) { + t.Run("Default config is valid", func(t *testing.T) { + err := createDefaultConfig().(*Config).Validate() + require.NoError(t, err) + }) + + t.Run("Empty configuration", func(t *testing.T) { + cfg := Config{ + Enabled: true, + Interval: defaultInterval, + AccountID: "myacct", + OrganizationID: "myorg", + } + err := cfg.Validate() + require.Error(t, err) + }) + + t.Run("Empty AccountID", func(t *testing.T) { + cfg := Config{ + Enabled: true, + Interval: defaultInterval, + OrganizationID: "myorg", + Configuration: "myconfig", + } + err := cfg.Validate() + require.Error(t, err) + }) + + t.Run("Empty OrganizationID", func(t *testing.T) { + cfg := Config{ + Enabled: true, + Interval: defaultInterval, + AccountID: "myacct", + Configuration: "myconfig", + } + err := cfg.Validate() + require.Error(t, err) + }) +} diff --git a/processor/topologyprocessor/factory.go b/processor/topologyprocessor/factory.go new file mode 100644 index 000000000..0592974b8 --- /dev/null +++ b/processor/topologyprocessor/factory.go @@ -0,0 +1,144 @@ +// Copyright observIQ, 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. + +package topologyprocessor + +import ( + "context" + "fmt" + "sync" + + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/processor" + "go.opentelemetry.io/collector/processor/processorhelper" +) + +var componentType = component.MustNewType("topology") + +const ( + stability = component.StabilityLevelAlpha +) + +var consumerCapabilities = consumer.Capabilities{MutatesData: false} + +// NewFactory creates a new ProcessorFactory with default configuration +func NewFactory() processor.Factory { + return processor.NewFactory( + componentType, + createDefaultConfig, + processor.WithTraces(createTracesProcessor, stability), + processor.WithLogs(createLogsProcessor, stability), + processor.WithMetrics(createMetricsProcessor, stability), + ) +} + +func createDefaultConfig() component.Config { + return &Config{ + Enabled: false, + Interval: defaultInterval, + } +} + +func createTracesProcessor( + ctx context.Context, + set processor.Settings, + cfg component.Config, + nextConsumer consumer.Traces, +) (processor.Traces, error) { + oCfg := cfg.(*Config) + tp, err := createOrGetProcessor(set, oCfg) + if err != nil { + return nil, fmt.Errorf("create topologyprocessor: %w", err) + } + + return processorhelper.NewTraces( + ctx, set, cfg, nextConsumer, tp.processTraces, + processorhelper.WithCapabilities(consumerCapabilities), + processorhelper.WithStart(tp.start), + processorhelper.WithShutdown(tp.shutdown), + ) +} + +func createLogsProcessor( + ctx context.Context, + set processor.Settings, + cfg component.Config, + nextConsumer consumer.Logs, +) (processor.Logs, error) { + oCfg := cfg.(*Config) + tp, err := createOrGetProcessor(set, oCfg) + if err != nil { + return nil, fmt.Errorf("create topologyprocessor: %w", err) + } + + return processorhelper.NewLogs( + ctx, set, cfg, nextConsumer, tp.processLogs, + processorhelper.WithCapabilities(consumerCapabilities), + processorhelper.WithStart(tp.start), + processorhelper.WithShutdown(tp.shutdown), + ) +} + +func createMetricsProcessor( + ctx context.Context, + set processor.Settings, + cfg component.Config, + nextConsumer consumer.Metrics, +) (processor.Metrics, error) { + oCfg := cfg.(*Config) + tp, err := createOrGetProcessor(set, oCfg) + if err != nil { + return nil, fmt.Errorf("create topologyprocessor: %w", err) + } + + return processorhelper.NewMetrics( + ctx, set, cfg, nextConsumer, tp.processMetrics, + processorhelper.WithCapabilities(consumerCapabilities), + processorhelper.WithStart(tp.start), + processorhelper.WithShutdown(tp.shutdown), + ) +} + +func createOrGetProcessor(set processor.Settings, cfg *Config) (*topologyProcessor, error) { + processorsMux.Lock() + defer processorsMux.Unlock() + + var tp *topologyProcessor + if p, ok := processors[set.ID]; ok { + tp = p + } else { + var err error + tp, err = newTopologyProcessor(set.Logger, cfg, set.ID) + if err != nil { + return nil, err + } + + processors[set.ID] = tp + } + + return tp, nil +} + +func unregisterProcessor(id component.ID) { + processorsMux.Lock() + defer processorsMux.Unlock() + delete(processors, id) +} + +// processors is a map of component.ID to an instance of topology processor. +// It is used so that only one instance of a particular topology processor exists, even if it's included +// across multiple pipelines/signal types. +var processors = map[component.ID]*topologyProcessor{} +var processorsMux = sync.Mutex{} diff --git a/processor/topologyprocessor/factory_test.go b/processor/topologyprocessor/factory_test.go new file mode 100644 index 000000000..83cae3214 --- /dev/null +++ b/processor/topologyprocessor/factory_test.go @@ -0,0 +1,193 @@ +// Copyright observIQ, 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. + +package topologyprocessor + +import ( + "context" + "testing" + + "github.com/observiq/bindplane-otel-collector/internal/topology" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/consumer/consumertest" + "go.opentelemetry.io/collector/processor/processortest" +) + +func TestNewFactory(t *testing.T) { + factory := NewFactory() + require.Equal(t, componentType, factory.Type()) + + expectedCfg := &Config{ + Enabled: false, + Interval: defaultInterval, + } + + cfg, ok := factory.CreateDefaultConfig().(*Config) + require.True(t, ok) + require.Equal(t, expectedCfg, cfg) +} + +func TestCreateOrGetProcessor(t *testing.T) { + p1Settings := processortest.NewNopSettings() + p1Settings.ID = component.MustNewIDWithName(componentType.String(), "proc1") + + p1, err := createOrGetProcessor(p1Settings, createDefaultConfig().(*Config)) + require.NoError(t, err) + p1Copy, err := createOrGetProcessor(p1Settings, createDefaultConfig().(*Config)) + require.NoError(t, err) + + // p1 and p1Copy should be the same pointer + require.True(t, p1 == p1Copy, "p1 and p1Copy are not the same pointer") + + p2Settings := processortest.NewNopSettings() + p2Settings.ID = component.MustNewIDWithName(componentType.String(), "proc2") + + p2, err := createOrGetProcessor(p2Settings, createDefaultConfig().(*Config)) + require.NoError(t, err) + require.True(t, p2 != p1, "p2 and p1 are the same, but they should be different objects") +} + +// Test that 2 instances with the same processor ID will not error when started +func TestCreateProcessorTwice_Logs(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + bindplaneExtensionID := component.MustNewID("bindplane") + + set := processortest.NewNopSettings() + set.ID = processorID + + cfg := &Config{ + Enabled: true, + Interval: defaultInterval, + Configuration: "myConf", + AccountID: "myAcct", + OrganizationID: "myOrg", + BindplaneExtension: bindplaneExtensionID, + } + + l1, err := createLogsProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + l2, err := createLogsProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + + mockBindplane := mockTopologyRegistry{ + ResettableTopologyRegistry: topology.NewResettableTopologyRegistry(), + } + + mh := mockHost{ + extMap: map[component.ID]component.Component{ + bindplaneExtensionID: mockBindplane, + }, + } + + require.NoError(t, l1.Start(context.Background(), mh)) + require.NoError(t, l2.Start(context.Background(), mh)) + require.NoError(t, l1.Shutdown(context.Background())) + require.NoError(t, l2.Shutdown(context.Background())) +} + +// Test that 2 instances with the same processor ID will not error when started +func TestCreateProcessorTwice_Metrics(t *testing.T) { + processorID := component.MustNewIDWithName("throughputmeasurement", "1") + bindplaneExtensionID := component.MustNewID("bindplane") + + set := processortest.NewNopSettings() + set.ID = processorID + + cfg := &Config{ + Enabled: true, + Interval: defaultInterval, + Configuration: "myConf", + AccountID: "myAcct", + OrganizationID: "myOrg", + BindplaneExtension: bindplaneExtensionID, + } + + l1, err := createMetricsProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + l2, err := createMetricsProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + + mockBindplane := mockTopologyRegistry{ + ResettableTopologyRegistry: topology.NewResettableTopologyRegistry(), + } + + mh := mockHost{ + extMap: map[component.ID]component.Component{ + bindplaneExtensionID: mockBindplane, + }, + } + + require.NoError(t, l1.Start(context.Background(), mh)) + require.NoError(t, l2.Start(context.Background(), mh)) + require.NoError(t, l1.Shutdown(context.Background())) + require.NoError(t, l2.Shutdown(context.Background())) +} + +// Test that 2 instances with the same processor ID will not error when started +func TestCreateProcessorTwice_Traces(t *testing.T) { + processorID := component.MustNewIDWithName("throughputmeasurement", "1") + bindplaneExtensionID := component.MustNewID("bindplane") + + set := processortest.NewNopSettings() + set.ID = processorID + + cfg := &Config{ + Enabled: true, + Interval: defaultInterval, + Configuration: "myConf", + AccountID: "myAcct", + OrganizationID: "myOrg", + BindplaneExtension: bindplaneExtensionID, + } + + l1, err := createTracesProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + l2, err := createTracesProcessor(context.Background(), set, cfg, consumertest.NewNop()) + require.NoError(t, err) + + mockBindplane := mockTopologyRegistry{ + ResettableTopologyRegistry: topology.NewResettableTopologyRegistry(), + } + + mh := mockHost{ + extMap: map[component.ID]component.Component{ + bindplaneExtensionID: mockBindplane, + }, + } + + require.NoError(t, l1.Start(context.Background(), mh)) + require.NoError(t, l2.Start(context.Background(), mh)) + require.NoError(t, l1.Shutdown(context.Background())) + require.NoError(t, l2.Shutdown(context.Background())) +} + +type mockHost struct { + extMap map[component.ID]component.Component +} + +func (m *mockHost) GetFactory(component.Kind, component.Type) component.Factory { + return nil +} + +func (m mockHost) GetExtensions() map[component.ID]component.Component { + return m.extMap +} + +type mockTopologyRegistry struct { + *topology.ResettableTopologyRegistry +} + +func (mockTopologyRegistry) Start(_ context.Context, _ component.Host) error { return nil } +func (mockTopologyRegistry) Shutdown(_ context.Context) error { return nil } diff --git a/processor/topologyprocessor/go.mod b/processor/topologyprocessor/go.mod new file mode 100644 index 000000000..f87c9284c --- /dev/null +++ b/processor/topologyprocessor/go.mod @@ -0,0 +1,58 @@ +module github.com/observiq/bindplane-otel-collector/processor/topologyprocessor + +go 1.22.6 + +require ( + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.116.0 + github.com/stretchr/testify v1.10.0 + go.opentelemetry.io/collector/component v0.116.0 + go.opentelemetry.io/collector/consumer v1.22.0 + go.opentelemetry.io/collector/consumer/consumertest v0.116.0 + go.opentelemetry.io/collector/pdata v1.22.0 + go.opentelemetry.io/collector/processor v0.116.0 + go.opentelemetry.io/collector/processor/processortest v0.116.0 + go.uber.org/zap v1.27.0 + google.golang.org/grpc v1.68.1 +) + +require ( + go.opentelemetry.io/collector/component/componenttest v0.116.0 // indirect + go.opentelemetry.io/collector/consumer/xconsumer v0.116.0 // indirect + go.opentelemetry.io/collector/processor/xprocessor v0.116.0 // indirect +) + +require ( + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect + github.com/go-logr/logr v1.4.2 // indirect + github.com/go-logr/stdr v1.2.2 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/google/uuid v1.6.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/observiq/bindplane-otel-collector/internal/topology v1.68.0 + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.116.0 + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0 // indirect + github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 // indirect + github.com/rogpeppe/go-internal v1.11.0 // indirect + go.opentelemetry.io/collector/component/componentstatus v0.116.0 // indirect + go.opentelemetry.io/collector/config/configtelemetry v0.116.0 // indirect + go.opentelemetry.io/collector/pdata/pprofile v0.116.0 // indirect + go.opentelemetry.io/collector/pdata/testdata v0.116.0 // indirect + go.opentelemetry.io/collector/pipeline v0.116.0 // indirect + go.opentelemetry.io/otel v1.32.0 // indirect + go.opentelemetry.io/otel/metric v1.32.0 // indirect + go.opentelemetry.io/otel/sdk v1.32.0 // indirect + go.opentelemetry.io/otel/sdk/metric v1.32.0 // indirect + go.opentelemetry.io/otel/trace v1.32.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + golang.org/x/net v0.29.0 // indirect + golang.org/x/sys v0.27.0 // indirect + golang.org/x/text v0.18.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 // indirect + google.golang.org/protobuf v1.35.2 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) + +replace github.com/observiq/bindplane-otel-collector/internal/topology => ../../internal/topology diff --git a/processor/topologyprocessor/go.sum b/processor/topologyprocessor/go.sum new file mode 100644 index 000000000..ec9eb65e3 --- /dev/null +++ b/processor/topologyprocessor/go.sum @@ -0,0 +1,136 @@ +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc h1:U9qPSI2PIWSS1VwoXQT9A3Wy9MM3WgvqSxFWenqJduM= +github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= +github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.116.0 h1:YENvOsl67sj8Ovvl5R8hKMnpPvdW3q5B7+CYYgy/GvQ= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden v0.116.0/go.mod h1:D56LJWVbMc1Kdy7qa6HCrHH6ZOr4yr7YuVfp1rJn0es= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.116.0 h1:RlEK9MbxWyBHbLel8EJ1L7DbYVLai9dZL6Ljl2cBgyA= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest v0.116.0/go.mod h1:AVUEyIjPb+0ARr7mhIkZkdNg3fd0ZcRhzAi53oZhl1Q= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0 h1:jwnZYRBuPJnsKXE5H6ZvTEm91bXW5VP8+tLewzl54eg= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatautil v0.116.0/go.mod h1:NT3Ag+DdnIAZQfD7l7OHwlYqnaAJ19SoPZ0nhD9yx4s= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2 h1:Jamvg5psRIccs7FGNTlIRMkT8wgtp5eCXdBlqhYGL6U= +github.com/pmezard/go-difflib v1.0.1-0.20181226105442-5d4384ee4fb2/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= +github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opentelemetry.io/collector/component v0.116.0 h1:SQE1YeVfYCN7bw1n4hknUwJE5U/1qJL552sDhAdSlaA= +go.opentelemetry.io/collector/component v0.116.0/go.mod h1:MYgXFZWDTq0uPgF1mkLSFibtpNqksRVAOrmihckOQEs= +go.opentelemetry.io/collector/component/componentstatus v0.116.0 h1:wpgY0H2K9IPBzaNAvavKziK86VZ7TuNFQbS9OC4Z6Cs= +go.opentelemetry.io/collector/component/componentstatus v0.116.0/go.mod h1:ZRlVwHFMGNfcsAywEJqivOn5JzDZkpe3KZVSwMWu4tw= +go.opentelemetry.io/collector/component/componenttest v0.116.0 h1:UIcnx4Rrs/oDRYSAZNHRMUiYs2FBlwgV5Nc0oMYfR6A= +go.opentelemetry.io/collector/component/componenttest v0.116.0/go.mod h1:W40HaKPHdBFMVI7zzHE7dhdWC+CgAnAC9SmWetFBATY= +go.opentelemetry.io/collector/config/configtelemetry v0.116.0 h1:Vl49VCHQwBOeMswDpFwcl2HD8e9y94xlrfII3SR2VeQ= +go.opentelemetry.io/collector/config/configtelemetry v0.116.0/go.mod h1:SlBEwQg0qly75rXZ6W1Ig8jN25KBVBkFIIAUI1GiAAE= +go.opentelemetry.io/collector/consumer v1.22.0 h1:QmfnNizyNZFt0uK3GG/EoT5h6PvZJ0dgVTc5hFEc1l0= +go.opentelemetry.io/collector/consumer v1.22.0/go.mod h1:tiz2khNceFAPokxxfzAuFfIpShBasMT2AL2Sbc7+m0I= +go.opentelemetry.io/collector/consumer/consumertest v0.116.0 h1:pIVR7FtQMNAzfxBUSMEIC2dX5Lfo3O9ZBfx+sAwrrrM= +go.opentelemetry.io/collector/consumer/consumertest v0.116.0/go.mod h1:cV3cNDiPnls5JdhnOJJFVlclrClg9kPs04cXgYP9Gmk= +go.opentelemetry.io/collector/consumer/xconsumer v0.116.0 h1:ZrWvq7HumB0jRYmS2ztZ3hhXRNpUVBWPKMbPhsVGmZM= +go.opentelemetry.io/collector/consumer/xconsumer v0.116.0/go.mod h1:C+VFMk8vLzPun6XK8aMts6h4RaDjmzXHCPaiOxzRQzQ= +go.opentelemetry.io/collector/pdata v1.22.0 h1:3yhjL46NLdTMoP8rkkcE9B0pzjf2973crn0KKhX5UrI= +go.opentelemetry.io/collector/pdata v1.22.0/go.mod h1:nLLf6uDg8Kn5g3WNZwGyu8+kf77SwOqQvMTb5AXEbEY= +go.opentelemetry.io/collector/pdata/pprofile v0.116.0 h1:iE6lqkO7Hi6lTIIml1RI7yQ55CKqW12R2qHinwF5Zuk= +go.opentelemetry.io/collector/pdata/pprofile v0.116.0/go.mod h1:xQiPpjzIiXRFb+1fPxUy/3ygEZgo0Bu/xmLKOWu8vMQ= +go.opentelemetry.io/collector/pdata/testdata v0.116.0 h1:zmn1zpeX2BvzL6vt2dBF4OuAyFF2ml/OXcqflNgFiP0= +go.opentelemetry.io/collector/pdata/testdata v0.116.0/go.mod h1:ytWzICFN4XTDP6o65B4+Ed52JGdqgk9B8CpLHCeCpMo= +go.opentelemetry.io/collector/pipeline v0.116.0 h1:o8eKEuWEszmRpfShy7ElBoQ3Jo6kCi9ucm3yRgdNb9s= +go.opentelemetry.io/collector/pipeline v0.116.0/go.mod h1:qE3DmoB05AW0C3lmPvdxZqd/H4po84NPzd5MrqgtL74= +go.opentelemetry.io/collector/processor v0.116.0 h1:Kyu4tPzTdWNHtZjcxvI/bGNAgyv8L8Kem2r/Mk4IDAw= +go.opentelemetry.io/collector/processor v0.116.0/go.mod h1:+/Ugy48RAxlZEXmN2cw51W8t5wdHS9No+GAoP+moskk= +go.opentelemetry.io/collector/processor/processortest v0.116.0 h1:+IqNEVEE0E2MsO2g7+Y/9dz35sDuvAXRXrLts9NdXrA= +go.opentelemetry.io/collector/processor/processortest v0.116.0/go.mod h1:DLaQDBxzgeeaUO0ULMn/efos9PmHZkmYCHuxwCsiVHI= +go.opentelemetry.io/collector/processor/xprocessor v0.116.0 h1:iin/UwuWvSLB7ZNfINFUYbZ5lxIi1NjZ2brkyyFdiRA= +go.opentelemetry.io/collector/processor/xprocessor v0.116.0/go.mod h1:cnA43/XpKDbaOmd8buqKp/LGJ2l/OoCqbR//u5DMfn8= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= +go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= +go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +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-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.29.0 h1:5ORfpBpCs4HzDYoodCDBbwHzdR5UrLBZ3sOnUJmFoHo= +golang.org/x/net v0.29.0/go.mod h1:gLkgy8jTGERgjzMic6DS9+SP0ajcu6Xu3Orq/SpETg0= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.27.0 h1:wBqf8DvsY9Y/2P8gAfPDEYNuS30J4lPHJxXSb/nJZ+s= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.18.0 h1:XvMDiNzPAl0jr17s6W9lcaIhGUfUORdGCNsuLmPG224= +golang.org/x/text v0.18.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1 h1:pPJltXNxVzT4pK9yD8vR9X75DaWYYmLGMsEvBfFQZzQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240903143218-8af14fe29dc1/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= +google.golang.org/grpc v1.68.1 h1:oI5oTa11+ng8r8XMMN7jAOmWfPZWbYpCFaMUTACxkM0= +google.golang.org/grpc v1.68.1/go.mod h1:+q1XYFJjShcqn0QZHvCyeR4CXPA+llXIeUIfIe00waw= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/processor/topologyprocessor/ocb_registry.go b/processor/topologyprocessor/ocb_registry.go new file mode 100644 index 000000000..78f30d8e8 --- /dev/null +++ b/processor/topologyprocessor/ocb_registry.go @@ -0,0 +1,46 @@ +// Copyright observIQ, 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. + +//go:build !bindplane + +package topologyprocessor + +import ( + "fmt" + + "github.com/observiq/bindplane-otel-collector/internal/topology" + "go.opentelemetry.io/collector/component" +) + +// GetTopologyRegistry returns the topology registry that should be registered to based on the component ID. +// nil, nil may be returned by this function. In this case, the processor should not register it's topology state anywhere. +func GetTopologyRegistry(host component.Host, bindplane component.ID) (topology.TopoRegistry, error) { + var emptyComponentID component.ID + if bindplane == emptyComponentID { + // No bindplane component referenced, so we won't register our topology state anywhere. + return nil, nil + } + + ext, ok := host.GetExtensions()[bindplane] + if !ok { + return nil, fmt.Errorf("bindplane extension %q does not exist", bindplane) + } + + registry, ok := ext.(topology.TopoRegistry) + if !ok { + return nil, fmt.Errorf("extension %q is not an topology state registry", bindplane) + } + + return registry, nil +} diff --git a/processor/topologyprocessor/processor.go b/processor/topologyprocessor/processor.go new file mode 100644 index 000000000..04f025357 --- /dev/null +++ b/processor/topologyprocessor/processor.go @@ -0,0 +1,155 @@ +// Copyright observIQ, 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. + +package topologyprocessor + +import ( + "context" + "fmt" + "strings" + "sync" + "time" + + "github.com/observiq/bindplane-otel-collector/internal/topology" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/pdata/plog" + "go.opentelemetry.io/collector/pdata/pmetric" + "go.opentelemetry.io/collector/pdata/ptrace" + "go.uber.org/zap" + "google.golang.org/grpc/metadata" +) + +const ( + organizationIDHeader = "X-Bindplane-Organization-ID" + accountIDHeader = "X-Bindplane-Account-ID" + configurationHeader = "X-Bindplane-Configuration" + resourceNameHeader = "X-Bindplane-Resource-Name" +) + +type topologyUpdate struct { + gw topology.GatewayInfo + routeTable map[topology.GatewayInfo]time.Time +} + +type topologyProcessor struct { + logger *zap.Logger + enabled bool + topology *topology.TopoState + interval time.Duration + processorID component.ID + bindplane component.ID + + startOnce sync.Once +} + +// newTopologyProcessor creates a new topology processor +func newTopologyProcessor(logger *zap.Logger, cfg *Config, processorID component.ID) (*topologyProcessor, error) { + destGw := topology.GatewayInfo{ + GatewayID: strings.TrimPrefix(processorID.String(), "topology/"), + Configuration: cfg.Configuration, + AccountID: cfg.AccountID, + OrganizationID: cfg.OrganizationID, + } + topology, err := topology.NewTopologyState(destGw) + if err != nil { + return nil, fmt.Errorf("create topology state: %w", err) + } + + return &topologyProcessor{ + logger: logger, + topology: topology, + processorID: processorID, + interval: cfg.Interval, + startOnce: sync.Once{}, + }, nil +} + +func (tp *topologyProcessor) start(_ context.Context, host component.Host) error { + var err error + tp.startOnce.Do(func() { + registry, getRegErr := GetTopologyRegistry(host, tp.bindplane) + if getRegErr != nil { + err = fmt.Errorf("get topology registry: %w", getRegErr) + return + } + + if registry != nil { + registerErr := registry.RegisterTopologyState(tp.processorID.String(), tp.topology) + if registerErr != nil { + return + } + registry.SetIntervalChan() <- tp.interval + } + }) + + return err +} + +func (tp *topologyProcessor) processTraces(ctx context.Context, td ptrace.Traces) (ptrace.Traces, error) { + tp.processTopologyHeaders(ctx) + return td, nil +} + +func (tp *topologyProcessor) processLogs(ctx context.Context, ld plog.Logs) (plog.Logs, error) { + tp.processTopologyHeaders(ctx) + return ld, nil +} + +func (tp *topologyProcessor) processMetrics(ctx context.Context, md pmetric.Metrics) (pmetric.Metrics, error) { + tp.processTopologyHeaders(ctx) + return md, nil +} + +func (tp *topologyProcessor) processTopologyHeaders(ctx context.Context) { + metadata, ok := metadata.FromIncomingContext(ctx) + if ok { + var configuration, accountID, organizationID, resourceName string + + configurationHeaders := metadata.Get(configurationHeader) + if len(configurationHeaders) > 0 { + configuration = configurationHeaders[0] + } + + accountIDHeaders := metadata.Get(accountIDHeader) + if len(accountIDHeaders) > 0 { + accountID = accountIDHeaders[0] + } + + organizationIDHeaders := metadata.Get(organizationIDHeader) + if len(organizationIDHeaders) > 0 { + organizationID = organizationIDHeaders[0] + } + + resourceNameHeaders := metadata.Get(resourceNameHeader) + if len(resourceNameHeaders) > 0 { + resourceName = resourceNameHeaders[0] + } + + // only upsert if all headers are present + if configuration != "" && accountID != "" && organizationID != "" && resourceName != "" { + gw := topology.GatewayInfo{ + Configuration: configuration, + AccountID: accountID, + OrganizationID: organizationID, + GatewayID: resourceName, + } + tp.topology.UpsertRoute(ctx, gw) + } + } +} + +func (tp *topologyProcessor) shutdown(_ context.Context) error { + unregisterProcessor(tp.processorID) + return nil +} diff --git a/processor/topologyprocessor/processor_test.go b/processor/topologyprocessor/processor_test.go new file mode 100644 index 000000000..c87296db6 --- /dev/null +++ b/processor/topologyprocessor/processor_test.go @@ -0,0 +1,221 @@ +// Copyright observIQ, 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. + +package topologyprocessor + +import ( + "context" + "path/filepath" + "testing" + "time" + + "github.com/observiq/bindplane-otel-collector/internal/topology" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/golden" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/plogtest" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/pmetrictest" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/pdatatest/ptracetest" + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/component" + "go.uber.org/zap" + "google.golang.org/grpc/metadata" +) + +func TestProcessor_Logs(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + + tmp, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID", + AccountID: "myAccountID", + Configuration: "myConfigName", + }, processorID) + require.NoError(t, err) + + logs, err := golden.ReadLogs(filepath.Join("testdata", "logs", "w3c-logs.yaml")) + require.NoError(t, err) + + ctx := metadata.NewIncomingContext(context.Background(), metadata.MD{ + accountIDHeader: []string{"myAccountID1"}, + organizationIDHeader: []string{"myOrgID1"}, + configurationHeader: []string{"myConfigName1"}, + resourceNameHeader: []string{"myResourceName1"}, + }) + processedLogs, err := tmp.processLogs(ctx, logs) + require.NoError(t, err) + + // Output logs should be the same as input logs (passthrough check) + require.NoError(t, plogtest.CompareLogs(logs, processedLogs)) + + // validate that upsert route was performed + require.True(t, tmp.topology.Topology.GatewaySource.AccountID == "myAccountID") + require.True(t, tmp.topology.Topology.GatewaySource.OrganizationID == "myOrgID") + require.True(t, tmp.topology.Topology.GatewaySource.Configuration == "myConfigName") + ci := topology.GatewayInfo{ + Configuration: "myConfigName1", + AccountID: "myAccountID1", + OrganizationID: "myOrgID1", + GatewayID: "myResourceName1", + } + _, ok := tmp.topology.Topology.RouteTable[ci] + require.True(t, ok) +} + +func TestProcessor_Metrics(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + + tmp, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID", + AccountID: "myAccountID", + Configuration: "myConfigName", + }, processorID) + require.NoError(t, err) + + metrics, err := golden.ReadMetrics(filepath.Join("testdata", "metrics", "host-metrics.yaml")) + require.NoError(t, err) + + ctx := metadata.NewIncomingContext(context.Background(), metadata.MD{ + accountIDHeader: []string{"myAccountID1"}, + organizationIDHeader: []string{"myOrgID1"}, + configurationHeader: []string{"myConfigName1"}, + resourceNameHeader: []string{"myResourceName1"}, + }) + + processedMetrics, err := tmp.processMetrics(ctx, metrics) + require.NoError(t, err) + + // Output metrics should be the same as input logs (passthrough check) + require.NoError(t, pmetrictest.CompareMetrics(metrics, processedMetrics)) + + // validate that upsert route was performed + require.True(t, tmp.topology.Topology.GatewaySource.AccountID == "myAccountID") + require.True(t, tmp.topology.Topology.GatewaySource.OrganizationID == "myOrgID") + require.True(t, tmp.topology.Topology.GatewaySource.Configuration == "myConfigName") + ci := topology.GatewayInfo{ + Configuration: "myConfigName1", + AccountID: "myAccountID1", + OrganizationID: "myOrgID1", + GatewayID: "myResourceName1", + } + _, ok := tmp.topology.Topology.RouteTable[ci] + require.True(t, ok) +} + +func TestProcessor_Traces(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + + tmp, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID", + AccountID: "myAccountID", + Configuration: "myConfigName", + }, processorID) + require.NoError(t, err) + + traces, err := golden.ReadTraces(filepath.Join("testdata", "traces", "bindplane-traces.yaml")) + require.NoError(t, err) + + ctx := metadata.NewIncomingContext(context.Background(), metadata.MD{ + accountIDHeader: []string{"myAccountID1"}, + organizationIDHeader: []string{"myOrgID1"}, + configurationHeader: []string{"myConfigName1"}, + resourceNameHeader: []string{"myResourceName1"}, + }) + + processedTraces, err := tmp.processTraces(ctx, traces) + require.NoError(t, err) + + // Output traces should be the same as input logs (passthrough check) + require.NoError(t, ptracetest.CompareTraces(traces, processedTraces)) + + // validate that upsert route was performed + require.True(t, tmp.topology.Topology.GatewaySource.AccountID == "myAccountID") + require.True(t, tmp.topology.Topology.GatewaySource.OrganizationID == "myOrgID") + require.True(t, tmp.topology.Topology.GatewaySource.Configuration == "myConfigName") + ci := topology.GatewayInfo{ + Configuration: "myConfigName1", + AccountID: "myAccountID1", + OrganizationID: "myOrgID1", + GatewayID: "myResourceName1", + } + _, ok := tmp.topology.Topology.RouteTable[ci] + require.True(t, ok) +} + +// Test 2 instances with the same processor ID +func TestProcessor_Logs_TwoInstancesSameID(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + + tmp1, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID", + AccountID: "myAccountID", + Configuration: "myConfigName", + }, processorID) + require.NoError(t, err) + + tmp2, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID2", + AccountID: "myAccountID2", + Configuration: "myConfigName2", + }, processorID) + require.NoError(t, err) + + logs, err := golden.ReadLogs(filepath.Join("testdata", "logs", "w3c-logs.yaml")) + require.NoError(t, err) + + _, err = tmp1.processLogs(context.Background(), logs) + require.NoError(t, err) + + _, err = tmp2.processLogs(context.Background(), logs) + require.NoError(t, err) +} + +func TestProcessor_Logs_TwoInstancesDifferentID(t *testing.T) { + processorID := component.MustNewIDWithName("topology", "1") + processorID2 := component.MustNewIDWithName("topology", "2") + + tmp1, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID", + AccountID: "myAccountID", + Configuration: "myConfigName", + }, processorID) + require.NoError(t, err) + + tmp2, err := newTopologyProcessor(zap.NewNop(), &Config{ + Enabled: true, + Interval: time.Second, + OrganizationID: "myOrgID2", + AccountID: "myAccountID2", + Configuration: "myConfigName2", + }, processorID2) + require.NoError(t, err) + + logs, err := golden.ReadLogs(filepath.Join("testdata", "logs", "w3c-logs.yaml")) + require.NoError(t, err) + + _, err = tmp1.processLogs(context.Background(), logs) + require.NoError(t, err) + + _, err = tmp2.processLogs(context.Background(), logs) + require.NoError(t, err) +} diff --git a/processor/topologyprocessor/testdata/logs/w3c-logs.yaml b/processor/topologyprocessor/testdata/logs/w3c-logs.yaml new file mode 100644 index 000000000..74eb70c6e --- /dev/null +++ b/processor/topologyprocessor/testdata/logs/w3c-logs.yaml @@ -0,0 +1,505 @@ +resourceLogs: + - resource: + attributes: + - key: host.name + value: + stringValue: Brandons-Legit-Windows-PC-Not-From-Mac-I-Swear + - key: os.type + value: + stringValue: windows + schemaUrl: https://opentelemetry.io/schemas/1.6.1 + scopeLogs: + - logRecords: + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: sc-status + value: + stringValue: "501" + - key: s-sitename + value: + stringValue: red-server + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 128.68.153.13 + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.909660 + - key: cs-method + value: + stringValue: DELETE + observedTimeUnixNano: "1706632435103211000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + stringValue: "This is a string body" + observedTimeUnixNano: "1706632435103211000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + - key: unique-attribute + value: + stringValue: unique-value + body: + kvlistValue: + values: + - key: s-ip + value: + stringValue: 19.25.92.158 + - key: cs-method + value: + stringValue: DELETE + - key: sc-status + value: + stringValue: "500" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.920213 + - key: s-sitename + value: + stringValue: blue-server + - key: s-computername + value: + stringValue: my-host + observedTimeUnixNano: "1706632435103340000" + spanId: "" + traceId: "" + scope: {} + - resource: + attributes: + - key: host.name + value: + stringValue: Brandons-MBP + - key: os.type + value: + stringValue: darwin + schemaUrl: https://opentelemetry.io/schemas/1.6.1 + scopeLogs: + - logRecords: + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-ip + value: + stringValue: 209.117.47.210 + - key: cs-method + value: + stringValue: POST + - key: sc-status + value: + stringValue: "200" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.760240 + - key: s-sitename + value: + stringValue: stage-server + - key: s-computername + value: + stringValue: my-host + observedTimeUnixNano: "1706632434903385000" + spanId: "" + traceId: "" + - attributes: + - key: log.file.name + value: + stringValue: example.log + - key: log_type + value: + stringValue: w3c + body: + kvlistValue: + values: + - key: sc-status + value: + stringValue: "400" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.771780 + - key: s-sitename + value: + stringValue: blue-server + - key: s-computername + value: + stringValue: my-host + - key: s-ip + value: + stringValue: 128.68.153.13 + - key: cs-method + value: + stringValue: POST + observedTimeUnixNano: "1706632434904438000" + spanId: "" + traceId: "" + - attributes: + - key: log.file.name + value: + stringValue: example.log + - key: log_type + value: + stringValue: w3c + body: + kvlistValue: + values: + - key: sc-status + value: + stringValue: "101" + - key: s-sitename + value: + stringValue: production-server + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 99.160.143.72 + - key: cs-method + value: + stringValue: GET + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.782933 + observedTimeUnixNano: "1706632434904726000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.793415 + - key: s-sitename + value: + stringValue: dev-server + - key: s-computername + value: + stringValue: ice-box + - key: s-ip + value: + stringValue: 129.90.224.41 + - key: cs-method + value: + stringValue: GET + - key: sc-status + value: + stringValue: "409" + observedTimeUnixNano: "1706632434904962000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-sitename + value: + stringValue: red-server + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 99.160.143.72 + - key: cs-method + value: + stringValue: GET + - key: sc-status + value: + stringValue: "101" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.803941 + observedTimeUnixNano: "1706632434905371000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-sitename + value: + stringValue: stage-server + - key: s-computername + value: + stringValue: my-host + - key: s-ip + value: + stringValue: 87.222.90.184 + - key: cs-method + value: + stringValue: GET + - key: sc-status + value: + stringValue: "501" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.814813 + observedTimeUnixNano: "1706632434905657000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-computername + value: + stringValue: my-host + - key: s-ip + value: + stringValue: 21.36.93.8 + - key: cs-method + value: + stringValue: DELETE + - key: sc-status + value: + stringValue: "400" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.825889 + - key: s-sitename + value: + stringValue: stage-server + observedTimeUnixNano: "1706632434905918000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.838153 + - key: s-sitename + value: + stringValue: blue-server + - key: s-computername + value: + stringValue: ice-box + - key: s-ip + value: + stringValue: 154.205.58.188 + - key: cs-method + value: + stringValue: PUT + - key: sc-status + value: + stringValue: "409" + observedTimeUnixNano: "1706632434906183000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-sitename + value: + stringValue: dev-server + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.850696 + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 19.25.92.158 + - key: cs-method + value: + stringValue: GET + - key: sc-status + value: + stringValue: "409" + observedTimeUnixNano: "1706632434906304000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-ip + value: + stringValue: 99.160.143.72 + - key: cs-method + value: + stringValue: PUT + - key: sc-status + value: + stringValue: "409" + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.861951 + - key: s-sitename + value: + stringValue: dev-server + - key: s-computername + value: + stringValue: my-host + observedTimeUnixNano: "1706632434906407000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.874685 + - key: cs-method + value: + stringValue: DELETE + - key: sc-status + value: + stringValue: "404" + - key: s-sitename + value: + stringValue: production-server + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 127.0.0.1 + observedTimeUnixNano: "1706632434906539000" + spanId: "" + traceId: "" + - attributes: + - key: log.file.name + value: + stringValue: example.log + - key: log_type + value: + stringValue: w3c + body: + kvlistValue: + values: + - key: s-computername + value: + stringValue: your-host + - key: s-ip + value: + stringValue: 21.36.93.8 + - key: cs-method + value: + stringValue: POST + - key: sc-status + value: + stringValue: "101" + - key: s-sitename + value: + stringValue: red-server + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.885779 + observedTimeUnixNano: "1706632434906584000" + spanId: "" + traceId: "" + - attributes: + - key: log_type + value: + stringValue: w3c + - key: log.file.name + value: + stringValue: example.log + body: + kvlistValue: + values: + - key: s-sitename + value: + stringValue: stage-server + - key: s-computername + value: + stringValue: ice-box + - key: timestamp + value: + stringValue: 30-01-2024 16:33:54.897564 + - key: s-ip + value: + stringValue: 19.25.92.158 + - key: cs-method + value: + stringValue: POST + - key: sc-status + value: + stringValue: "409" + observedTimeUnixNano: "1706632434906598000" + spanId: "" + traceId: "" + scope: {} diff --git a/processor/topologyprocessor/testdata/metrics/host-metrics.yaml b/processor/topologyprocessor/testdata/metrics/host-metrics.yaml new file mode 100644 index 000000000..d42978181 --- /dev/null +++ b/processor/topologyprocessor/testdata/metrics/host-metrics.yaml @@ -0,0 +1,737 @@ +resourceMetrics: + - resource: + attributes: + - key: host.name + value: + stringValue: Brandons-Awesome-Linux-Machine + - key: os.type + value: + stringValue: linux + - key: extra-resource-attr-key + value: + stringValue: extra-resource-attr-value + schemaUrl: https://opentelemetry.io/schemas/1.9.0 + scopeMetrics: + - metrics: + - description: Average CPU Load over 1 minute. + gauge: + dataPoints: + - asDouble: 3.71484375 + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + attributes: + - key: cool-attribute-key + value: + stringValue: cool-attribute-value + name: system.cpu.load_average.1m + unit: "{thread}" + scope: + name: otelcol/hostmetricsreceiver/load + version: v1.43.0 + - resource: + attributes: + - key: host.name + value: + stringValue: Brandons-MBP + - key: os.type + value: + stringValue: darwin + schemaUrl: https://opentelemetry.io/schemas/1.9.0 + scopeMetrics: + - metrics: + - description: Filesystem bytes used. + name: system.filesystem.usage + sum: + aggregationTemporality: 2 + dataPoints: + - asInt: "503869440" + attributes: + - key: device + value: + stringValue: /dev/disk2s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/iSCPreboot + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk2s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/iSCPreboot + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "20418560" + attributes: + - key: device + value: + stringValue: /dev/disk2s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/iSCPreboot + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "503869440" + attributes: + - key: device + value: + stringValue: /dev/disk2s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/xarts + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk2s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/xarts + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "20418560" + attributes: + - key: device + value: + stringValue: /dev/disk2s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/xarts + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "503869440" + attributes: + - key: device + value: + stringValue: /dev/disk2s3 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Hardware + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk2s3 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Hardware + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "20418560" + attributes: + - key: device + value: + stringValue: /dev/disk2s3 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Hardware + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "342134198272" + attributes: + - key: device + value: + stringValue: /dev/disk3s1s1 + - key: mode + value: + stringValue: ro + - key: mountpoint + value: + stringValue: / + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk3s1s1 + - key: mode + value: + stringValue: ro + - key: mountpoint + value: + stringValue: / + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "152250597376" + attributes: + - key: device + value: + stringValue: /dev/disk3s1s1 + - key: mode + value: + stringValue: ro + - key: mountpoint + value: + stringValue: / + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "342134198272" + attributes: + - key: device + value: + stringValue: /dev/disk3s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Preboot + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk3s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Preboot + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "152250597376" + attributes: + - key: device + value: + stringValue: /dev/disk3s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Preboot + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "342134198272" + attributes: + - key: device + value: + stringValue: /dev/disk3s4 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Update + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk3s4 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Update + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "152250597376" + attributes: + - key: device + value: + stringValue: /dev/disk3s4 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Update + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "342134198272" + attributes: + - key: device + value: + stringValue: /dev/disk3s5 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk3s5 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "152250597376" + attributes: + - key: device + value: + stringValue: /dev/disk3s5 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "342134198272" + attributes: + - key: device + value: + stringValue: /dev/disk3s6 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/VM + - key: state + value: + stringValue: free + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk3s6 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/VM + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "152250597376" + attributes: + - key: device + value: + stringValue: /dev/disk3s6 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/VM + - key: state + value: + stringValue: used + - key: type + value: + stringValue: apfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "8717185024" + attributes: + - key: extra-sum-attr-key + value: + stringValue: extra-sum-attr-value + - key: device + value: + stringValue: /dev/disk4s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/transfer + - key: state + value: + stringValue: free + - key: type + value: + stringValue: hfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk4s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/transfer + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: hfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "7409389568" + attributes: + - key: device + value: + stringValue: /dev/disk4s1 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/transfer + - key: state + value: + stringValue: used + - key: type + value: + stringValue: hfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "14336" + attributes: + - key: device + value: + stringValue: /dev/disk4s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/UEFI_NTFS + - key: state + value: + stringValue: free + - key: type + value: + stringValue: msdos + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: /dev/disk4s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/UEFI_NTFS + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: msdos + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "493568" + attributes: + - key: device + value: + stringValue: /dev/disk4s2 + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /Volumes/UEFI_NTFS + - key: state + value: + stringValue: used + - key: type + value: + stringValue: msdos + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: devfs + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /dev + - key: state + value: + stringValue: free + - key: type + value: + stringValue: devfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: devfs + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /dev + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: devfs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "220672" + attributes: + - key: device + value: + stringValue: devfs + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /dev + - key: state + value: + stringValue: used + - key: type + value: + stringValue: devfs + startTimeUnixNano: "1000000" + timeUnixNano: "3000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: map auto_home + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data/home + - key: state + value: + stringValue: free + - key: type + value: + stringValue: autofs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: map auto_home + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data/home + - key: state + value: + stringValue: reserved + - key: type + value: + stringValue: autofs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + - asInt: "0" + attributes: + - key: device + value: + stringValue: map auto_home + - key: mode + value: + stringValue: rw + - key: mountpoint + value: + stringValue: /System/Volumes/Data/home + - key: state + value: + stringValue: used + - key: type + value: + stringValue: autofs + startTimeUnixNano: "1000000" + timeUnixNano: "2000000" + unit: By + scope: + name: otelcol/hostmetricsreceiver/filesystem + version: v1.43.0 diff --git a/processor/topologyprocessor/testdata/traces/bindplane-traces.yaml b/processor/topologyprocessor/testdata/traces/bindplane-traces.yaml new file mode 100644 index 000000000..e15905b84 --- /dev/null +++ b/processor/topologyprocessor/testdata/traces/bindplane-traces.yaml @@ -0,0 +1,1530 @@ +resourceSpans: + - resource: + attributes: + - key: host.arch + value: + stringValue: arm64 + - key: host.name + value: + stringValue: Sams-M1-Pro.local + - key: service.name + value: + stringValue: bindplane + - key: service.version + value: + stringValue: unknown + scopeSpans: + - scope: {} + spans: + - endTimeUnixNano: "1706791445370505958" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: 01f07757e7bb6612 + spanId: 5d85207073e1d06b + startTimeUnixNano: "1706791445369534000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445371594000" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 1e9c016ee3394bea + spanId: 39f855dc7dbd43ae + startTimeUnixNano: "1706791445371218000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445371594459" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: aa50d71d28f47370 + spanId: 1e9c016ee3394bea + startTimeUnixNano: "1706791445370893000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445371595166" + kind: 1 + name: pgstore/pgResource + parentSpanId: 560da73813316270 + spanId: aa50d71d28f47370 + startTimeUnixNano: "1706791445370893000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445371594708" + kind: 1 + name: pgstore/SourceType + parentSpanId: 0f3367e6b090ffed + spanId: 560da73813316270 + startTimeUnixNano: "1706791445370892000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445371595959" + kind: 1 + name: featuregatedstore/SourceType + parentSpanId: fd55f461239efdfc + spanId: 0f3367e6b090ffed + startTimeUnixNano: "1706791445370892000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445371621792" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 5aceab5faac84e72 + spanId: 34db33f306cd28d6 + startTimeUnixNano: "1706791445371252000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445371622334" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 96cebc89a09695f9 + spanId: 5aceab5faac84e72 + startTimeUnixNano: "1706791445370905000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445371621958" + kind: 1 + name: pgstore/pgResource + parentSpanId: ed9b4f8caac8c99e + spanId: 96cebc89a09695f9 + startTimeUnixNano: "1706791445370904000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445371622458" + kind: 1 + name: pgstore/SourceType + parentSpanId: ac5df0c7e93047f1 + spanId: ed9b4f8caac8c99e + startTimeUnixNano: "1706791445370904000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445371623000" + kind: 1 + name: featuregatedstore/SourceType + parentSpanId: 01f07757e7bb6612 + spanId: ac5df0c7e93047f1 + startTimeUnixNano: "1706791445370903000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372255042" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: ac1591195af28043 + spanId: 439f48f98419cd32 + startTimeUnixNano: "1706791445372193000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372256000" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 7a0d816f78afc130 + spanId: ac1591195af28043 + startTimeUnixNano: "1706791445371916000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372255708" + kind: 1 + name: pgstore/pgResource + parentSpanId: fa26c6c00e05b5eb + spanId: 7a0d816f78afc130 + startTimeUnixNano: "1706791445371915000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372255500" + kind: 1 + name: pgstore/Destination + parentSpanId: fd55f461239efdfc + spanId: fa26c6c00e05b5eb + startTimeUnixNano: "1706791445371914000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372285833" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: fd5ad671cccf90ac + spanId: bb03baad52e8e234 + startTimeUnixNano: "1706791445372232000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372286291" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 9362c54ca8d2781f + spanId: fd5ad671cccf90ac + startTimeUnixNano: "1706791445371921000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372286917" + kind: 1 + name: pgstore/pgResource + parentSpanId: 84b85b95e4e3302c + spanId: 9362c54ca8d2781f + startTimeUnixNano: "1706791445371921000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372286750" + kind: 1 + name: pgstore/Destination + parentSpanId: 01f07757e7bb6612 + spanId: 84b85b95e4e3302c + startTimeUnixNano: "1706791445371920000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372638125" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 5caae46954be319e + spanId: b064b57576599977 + startTimeUnixNano: "1706791445372515000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372639250" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 6a0757daee65f85d + spanId: 5caae46954be319e + startTimeUnixNano: "1706791445372258000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372639833" + kind: 1 + name: pgstore/pgResource + parentSpanId: 87ec389ec0a90e57 + spanId: 6a0757daee65f85d + startTimeUnixNano: "1706791445372258000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372639375" + kind: 1 + name: pgstore/DestinationType + parentSpanId: 3d1d65aa7a6498e2 + spanId: 87ec389ec0a90e57 + startTimeUnixNano: "1706791445372257000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372640333" + kind: 1 + name: featuregatedstore/DestinationType + parentSpanId: fd55f461239efdfc + spanId: 3d1d65aa7a6498e2 + startTimeUnixNano: "1706791445372257000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445372673791" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: ac718a9ec3ef8ce8 + spanId: 8759e7f478277358 + startTimeUnixNano: "1706791445372561000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372674042" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 547d3cefd8be11ba + spanId: ac718a9ec3ef8ce8 + startTimeUnixNano: "1706791445372289000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372674709" + kind: 1 + name: pgstore/pgResource + parentSpanId: cc75d9bcab91629f + spanId: 547d3cefd8be11ba + startTimeUnixNano: "1706791445372289000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372674208" + kind: 1 + name: pgstore/DestinationType + parentSpanId: 98bca1b4ce950e6a + spanId: cc75d9bcab91629f + startTimeUnixNano: "1706791445372288000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445372675000" + kind: 1 + name: featuregatedstore/DestinationType + parentSpanId: 01f07757e7bb6612 + spanId: 98bca1b4ce950e6a + startTimeUnixNano: "1706791445372288000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373307000" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 31427ef8f1cdb87e + spanId: 4c5347222741d80a + startTimeUnixNano: "1706791445373202000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373310333" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 623a6ef09a05d78e + spanId: 31427ef8f1cdb87e + startTimeUnixNano: "1706791445372919000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373310958" + kind: 1 + name: pgstore/pgResource + parentSpanId: eabea48cc976c26c + spanId: 623a6ef09a05d78e + startTimeUnixNano: "1706791445372919000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373310584" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: 59c6fbff32109911 + spanId: eabea48cc976c26c + startTimeUnixNano: "1706791445372918000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373311500" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: 01f07757e7bb6612 + spanId: 59c6fbff32109911 + startTimeUnixNano: "1706791445372918000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373317417" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 81d1379f833e45e4 + spanId: af9f808e0386dcc1 + startTimeUnixNano: "1706791445373215000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373318125" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 8c1b5f014af598db + spanId: 81d1379f833e45e4 + startTimeUnixNano: "1706791445372895000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373317791" + kind: 1 + name: pgstore/pgResource + parentSpanId: ec1965c48fa74dd8 + spanId: 8c1b5f014af598db + startTimeUnixNano: "1706791445372894000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373318500" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: 2be401733ad05f07 + spanId: ec1965c48fa74dd8 + startTimeUnixNano: "1706791445372894000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373318417" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: fd55f461239efdfc + spanId: 2be401733ad05f07 + startTimeUnixNano: "1706791445372893000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373741542" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 1073bab272ed8d99 + spanId: fe61a5e73fdfcd54 + startTimeUnixNano: "1706791445373681000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373742542" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 6482ba65fc5d6aea + spanId: 1073bab272ed8d99 + startTimeUnixNano: "1706791445373405000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373742291" + kind: 1 + name: pgstore/pgResource + parentSpanId: 8a7b5c2c9a64b49b + spanId: 6482ba65fc5d6aea + startTimeUnixNano: "1706791445373404000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373743042" + kind: 1 + name: pgstore/Destination + parentSpanId: fd55f461239efdfc + spanId: 8a7b5c2c9a64b49b + startTimeUnixNano: "1706791445373404000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445373755459" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 92b6c441dbaf26e7 + spanId: 73cf26fff96c6144 + startTimeUnixNano: "1706791445373702000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373756125" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: b765ba46b31ea2b7 + spanId: 92b6c441dbaf26e7 + startTimeUnixNano: "1706791445373395000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373756125" + kind: 1 + name: pgstore/pgResource + parentSpanId: edfdd26910042116 + spanId: b765ba46b31ea2b7 + startTimeUnixNano: "1706791445373394000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445373756042" + kind: 1 + name: pgstore/Destination + parentSpanId: 01f07757e7bb6612 + spanId: edfdd26910042116 + startTimeUnixNano: "1706791445373393000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445374104125" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: ee6eba613911c7f7 + spanId: 9838adef042dc27f + startTimeUnixNano: "1706791445373993000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445374105625" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 8fd1f88bb661dbbf + spanId: ee6eba613911c7f7 + startTimeUnixNano: "1706791445373745000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445374105292" + kind: 1 + name: pgstore/pgResource + parentSpanId: 4518fe3e7e061d28 + spanId: 8fd1f88bb661dbbf + startTimeUnixNano: "1706791445373744000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445374106000" + kind: 1 + name: pgstore/DestinationType + parentSpanId: dc8f0547997bbf7a + spanId: 4518fe3e7e061d28 + startTimeUnixNano: "1706791445373744000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445374106125" + kind: 1 + name: featuregatedstore/DestinationType + parentSpanId: fd55f461239efdfc + spanId: dc8f0547997bbf7a + startTimeUnixNano: "1706791445373743000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445374128917" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 93e5965556047830 + spanId: b3a494cb25e7968e + startTimeUnixNano: "1706791445374013000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445374128916" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 48e733575bb79f9a + spanId: 93e5965556047830 + startTimeUnixNano: "1706791445373758000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445374129583" + kind: 1 + name: pgstore/pgResource + parentSpanId: ab1f6cb09be6fd07 + spanId: 48e733575bb79f9a + startTimeUnixNano: "1706791445373758000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445374129084" + kind: 1 + name: pgstore/DestinationType + parentSpanId: 2a2838388f079090 + spanId: ab1f6cb09be6fd07 + startTimeUnixNano: "1706791445373757000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445374129917" + kind: 1 + name: featuregatedstore/DestinationType + parentSpanId: 01f07757e7bb6612 + spanId: 2a2838388f079090 + startTimeUnixNano: "1706791445373757000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445375289875" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 58eb2369bc29028b + spanId: 3d367cddad3982cb + startTimeUnixNano: "1706791445374690000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445375291042" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 8b48d61bb3bd84b5 + spanId: 58eb2369bc29028b + startTimeUnixNano: "1706791445374349000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445375291667" + kind: 1 + name: pgstore/pgResource + parentSpanId: ea1323fc9fa7bbfb + spanId: 8b48d61bb3bd84b5 + startTimeUnixNano: "1706791445374349000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445375291291" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: 883ad1808f1225df + spanId: ea1323fc9fa7bbfb + startTimeUnixNano: "1706791445374348000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445375292333" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: fd55f461239efdfc + spanId: 883ad1808f1225df + startTimeUnixNano: "1706791445374348000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445375292458" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 1324f3c06efff713 + spanId: 8b340a6af31402a6 + startTimeUnixNano: "1706791445374683000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445375293083" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 0389a1ce82df8073 + spanId: 1324f3c06efff713 + startTimeUnixNano: "1706791445374369000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445375292916" + kind: 1 + name: pgstore/pgResource + parentSpanId: fbaba0d15c7fb273 + spanId: 0389a1ce82df8073 + startTimeUnixNano: "1706791445374368000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445375293750" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: 2ec705763a482c63 + spanId: fbaba0d15c7fb273 + startTimeUnixNano: "1706791445374368000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445375293708" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: 01f07757e7bb6612 + spanId: 2ec705763a482c63 + startTimeUnixNano: "1706791445374367000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445376188917" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: aa9bc90319da6537 + spanId: d895d7bb7f1e1e22 + startTimeUnixNano: "1706791445376038000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445376190083" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: a5f8b395ad89fb20 + spanId: aa9bc90319da6537 + startTimeUnixNano: "1706791445375734000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445376190834" + kind: 1 + name: pgstore/pgResource + parentSpanId: cdff48ed0c3e72ae + spanId: a5f8b395ad89fb20 + startTimeUnixNano: "1706791445375734000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445376190500" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: b67f47ab8ceb6c2c + spanId: cdff48ed0c3e72ae + startTimeUnixNano: "1706791445375733000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445376191584" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: fd55f461239efdfc + spanId: b67f47ab8ceb6c2c + startTimeUnixNano: "1706791445375733000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - endTimeUnixNano: "1706791445376214000" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: eff5bf5ccd3ab227 + spanId: d501cddc5c10bb68 + startTimeUnixNano: "1706791445376044000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445376214125" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 3e9b9f81e55ccd89 + spanId: eff5bf5ccd3ab227 + startTimeUnixNano: "1706791445375749000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445376213958" + kind: 1 + name: pgstore/pgResource + parentSpanId: 220b7c5429a375d4 + spanId: 3e9b9f81e55ccd89 + startTimeUnixNano: "1706791445375748000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445376214708" + kind: 1 + name: pgstore/ProcessorType + parentSpanId: 924c1623bafaba3b + spanId: 220b7c5429a375d4 + startTimeUnixNano: "1706791445375748000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445376214834" + kind: 1 + name: featuregatedstore/ProcessorType + parentSpanId: 01f07757e7bb6612 + spanId: 924c1623bafaba3b + startTimeUnixNano: "1706791445375747000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - endTimeUnixNano: "1706791445912660750" + kind: 1 + name: pgstore/getConfigurationIndex + parentSpanId: 9e9edb4efbb989ef + spanId: 18a40a19cbb3e254 + startTimeUnixNano: "1706791445912659000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445912667166" + kind: 1 + name: pgstore/ConfigurationIndex + parentSpanId: 9d5a7e824fa7ba3b + spanId: 9e9edb4efbb989ef + startTimeUnixNano: "1706791445912658000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445914167041" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: f58f50240bd0ebcb + spanId: b296ae6c001f640a + startTimeUnixNano: "1706791445913897000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445916449917" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: cce7e1ec6abeb8dc + spanId: 5f43117727ebc00e + startTimeUnixNano: "1706791445915074000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445916456958" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: f58f50240bd0ebcb + spanId: cce7e1ec6abeb8dc + startTimeUnixNano: "1706791445914332000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919373083" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: cc9471536eccd5db + spanId: 3536cf01c143e85f + startTimeUnixNano: "1706791445918350000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919379208" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: bc09affd2efa24ae + spanId: cc9471536eccd5db + startTimeUnixNano: "1706791445917392000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919379917" + kind: 1 + name: pgstore/pgResource + parentSpanId: 09ef41113f7d7845 + spanId: bc09affd2efa24ae + startTimeUnixNano: "1706791445917390000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919490917" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: f58f50240bd0ebcb + spanId: 09ef41113f7d7845 + startTimeUnixNano: "1706791445917387000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919633917" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: 5fc019ad71775af4 + spanId: aa56c92fbc435074 + startTimeUnixNano: "1706791445919632000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919634792" + kind: 1 + name: pgstore/notify + parentSpanId: f58f50240bd0ebcb + spanId: 5fc019ad71775af4 + startTimeUnixNano: "1706791445919629000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919906375" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: f58f50240bd0ebcb + spanId: 185ab719965503f6 + startTimeUnixNano: "1706791445919637000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445919909125" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: f58f50240bd0ebcb + startTimeUnixNano: "1706791445913891000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445920184208" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: 343173155818d664 + spanId: 61c28a1ebe9a64e8 + startTimeUnixNano: "1706791445919918000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445921631916" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: c85313d8e38e60c0 + spanId: 1db8a9899aca2909 + startTimeUnixNano: "1706791445920986000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445921637667" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 343173155818d664 + spanId: c85313d8e38e60c0 + startTimeUnixNano: "1706791445920299000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923689583" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: ea20cd55767a0599 + spanId: dee65f909bbc0d64 + startTimeUnixNano: "1706791445923120000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923695125" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 0cc1b95036e397be + spanId: ea20cd55767a0599 + startTimeUnixNano: "1706791445922423000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923696417" + kind: 1 + name: pgstore/pgResource + parentSpanId: 849c53c707b9f279 + spanId: 0cc1b95036e397be + startTimeUnixNano: "1706791445922422000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923777000" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: 343173155818d664 + spanId: 849c53c707b9f279 + startTimeUnixNano: "1706791445922419000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923905459" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: 845f46aa58bbcb19 + spanId: dead9c5716ba621d + startTimeUnixNano: "1706791445923904000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445923907417" + kind: 1 + name: pgstore/notify + parentSpanId: 343173155818d664 + spanId: 845f46aa58bbcb19 + startTimeUnixNano: "1706791445923902000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445924146459" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: 343173155818d664 + spanId: 17c792e43c54f830 + startTimeUnixNano: "1706791445923908000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445924148792" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: 343173155818d664 + startTimeUnixNano: "1706791445919913000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445924361000" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: 38ff7d679d77bdbd + spanId: b6e7f0ae7967d288 + startTimeUnixNano: "1706791445924159000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445925556834" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 48c2cef07ce74c65 + spanId: 83e3224e213642fa + startTimeUnixNano: "1706791445925082000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445925565458" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 38ff7d679d77bdbd + spanId: 48c2cef07ce74c65 + startTimeUnixNano: "1706791445924467000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927498625" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 6b63f670942e88f2 + spanId: aa9d17b7b3c4ea09 + startTimeUnixNano: "1706791445926821000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927501542" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 88ff93a450488c13 + spanId: 6b63f670942e88f2 + startTimeUnixNano: "1706791445926216000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927501625" + kind: 1 + name: pgstore/pgResource + parentSpanId: 1e96a9988bef1b05 + spanId: 88ff93a450488c13 + startTimeUnixNano: "1706791445926214000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927556459" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: 38ff7d679d77bdbd + spanId: 1e96a9988bef1b05 + startTimeUnixNano: "1706791445926212000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927702458" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: b6e1d82a58e8fd61 + spanId: 5a055056ad7713e5 + startTimeUnixNano: "1706791445927700000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445927704250" + kind: 1 + name: pgstore/notify + parentSpanId: 38ff7d679d77bdbd + spanId: b6e1d82a58e8fd61 + startTimeUnixNano: "1706791445927698000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445928022709" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: 38ff7d679d77bdbd + spanId: 1a79a3b2c24f43db + startTimeUnixNano: "1706791445927705000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445928025459" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: 38ff7d679d77bdbd + startTimeUnixNano: "1706791445924153000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445928253125" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: f667fcb4d9278ef3 + spanId: bd29d8d73527c437 + startTimeUnixNano: "1706791445928036000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445929417833" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: fc0c17bdf1248a4b + spanId: 850448ca72ab8c7e + startTimeUnixNano: "1706791445928966000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445929422500" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: f667fcb4d9278ef3 + spanId: fc0c17bdf1248a4b + startTimeUnixNano: "1706791445928344000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931223792" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: b4fdd461cf7cff72 + spanId: 3db6d250c2670120 + startTimeUnixNano: "1706791445930719000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931226000" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: c3b8a5e5232d4f97 + spanId: b4fdd461cf7cff72 + startTimeUnixNano: "1706791445930124000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931227292" + kind: 1 + name: pgstore/pgResource + parentSpanId: 9058a7f1ddbdb478 + spanId: c3b8a5e5232d4f97 + startTimeUnixNano: "1706791445930123000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931289667" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: f667fcb4d9278ef3 + spanId: 9058a7f1ddbdb478 + startTimeUnixNano: "1706791445930120000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931530167" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: 174ceb1bea9655e4 + spanId: 64bdb41cf5cf5488 + startTimeUnixNano: "1706791445931528000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931531541" + kind: 1 + name: pgstore/notify + parentSpanId: f667fcb4d9278ef3 + spanId: 174ceb1bea9655e4 + startTimeUnixNano: "1706791445931525000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931859917" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: f667fcb4d9278ef3 + spanId: a5d5cb6812b504eb + startTimeUnixNano: "1706791445931533000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445931865666" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: f667fcb4d9278ef3 + startTimeUnixNano: "1706791445928032000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445932189291" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: a502177e675181ba + spanId: 742646f2a069a533 + startTimeUnixNano: "1706791445931875000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445933524042" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: d8578a54be7b449a + spanId: af80a97174fb9690 + startTimeUnixNano: "1706791445933082000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445933525458" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: a502177e675181ba + spanId: d8578a54be7b449a + startTimeUnixNano: "1706791445932312000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445935873208" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 83a7c9b53a487359 + spanId: 35563160cfcceaed + startTimeUnixNano: "1706791445935438000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445935876667" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: b1f35deb03b68692 + spanId: 83a7c9b53a487359 + startTimeUnixNano: "1706791445934726000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445935877375" + kind: 1 + name: pgstore/pgResource + parentSpanId: d4cc9c9ef3675f58 + spanId: b1f35deb03b68692 + startTimeUnixNano: "1706791445934724000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445935973000" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: a502177e675181ba + spanId: d4cc9c9ef3675f58 + startTimeUnixNano: "1706791445934719000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445936170958" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: dc67925deaac8e96 + spanId: 67dc50cbcf1a0507 + startTimeUnixNano: "1706791445936169000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445936172167" + kind: 1 + name: pgstore/notify + parentSpanId: a502177e675181ba + spanId: dc67925deaac8e96 + startTimeUnixNano: "1706791445936167000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445937520417" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: a502177e675181ba + spanId: 78bbc878254952f1 + startTimeUnixNano: "1706791445936174000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445937527333" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: a502177e675181ba + startTimeUnixNano: "1706791445931870000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445937820041" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: a894c9beda2d173a + spanId: 942d1791aef3d492 + startTimeUnixNano: "1706791445937543000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445961888125" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: a87163b208c61459 + spanId: 0ccc6475795e4cc0 + startTimeUnixNano: "1706791445960465000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445961896542" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: a894c9beda2d173a + spanId: a87163b208c61459 + startTimeUnixNano: "1706791445937922000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973101000" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 804ce3fb1b57be5d + spanId: 3eadb90414f2cf22 + startTimeUnixNano: "1706791445971633000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973109084" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 5236aa938eb9341c + spanId: 804ce3fb1b57be5d + startTimeUnixNano: "1706791445962593000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973110334" + kind: 1 + name: pgstore/pgResource + parentSpanId: ca3b8e53681a9e8d + spanId: 5236aa938eb9341c + startTimeUnixNano: "1706791445962592000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973258875" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: a894c9beda2d173a + spanId: ca3b8e53681a9e8d + startTimeUnixNano: "1706791445962589000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973576917" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: 94557eef510d0814 + spanId: c3ea8f993f9009ba + startTimeUnixNano: "1706791445973575000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445973579333" + kind: 1 + name: pgstore/notify + parentSpanId: a894c9beda2d173a + spanId: 94557eef510d0814 + startTimeUnixNano: "1706791445973572000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445975115042" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: a894c9beda2d173a + spanId: 8906c43abedb6bd9 + startTimeUnixNano: "1706791445973581000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445975119416" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: a894c9beda2d173a + startTimeUnixNano: "1706791445937536000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445975734417" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: 078ab0ab7eb707b2 + spanId: f141ff98614cfc0c + startTimeUnixNano: "1706791445975133000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445979767041" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: f06908731ed62890 + spanId: e185a7e1c60473b8 + startTimeUnixNano: "1706791445976351000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445979799208" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 078ab0ab7eb707b2 + spanId: f06908731ed62890 + startTimeUnixNano: "1706791445975818000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445989063500" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: 484272979ab6993a + spanId: f0e946762f0f7fe4 + startTimeUnixNano: "1706791445982071000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445989071125" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 1f3d24377d56ef1a + spanId: 484272979ab6993a + startTimeUnixNano: "1706791445981428000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445989073000" + kind: 1 + name: pgstore/pgResource + parentSpanId: f2caaa5dba4d7692 + spanId: 1f3d24377d56ef1a + startTimeUnixNano: "1706791445981427000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445995086875" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: 078ab0ab7eb707b2 + spanId: f2caaa5dba4d7692 + startTimeUnixNano: "1706791445981424000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445995678750" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: d2cd0bbf6e7fd7b6 + spanId: 0f0e2864b0289733 + startTimeUnixNano: "1706791445995677000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445995680917" + kind: 1 + name: pgstore/notify + parentSpanId: 078ab0ab7eb707b2 + spanId: d2cd0bbf6e7fd7b6 + startTimeUnixNano: "1706791445995673000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445996202250" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: 078ab0ab7eb707b2 + spanId: 6a342964854bdad1 + startTimeUnixNano: "1706791445995682000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445996208000" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 9d5a7e824fa7ba3b + spanId: 078ab0ab7eb707b2 + startTimeUnixNano: "1706791445975127000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445996209292" + kind: 1 + name: pgstore/UpdateRollouts + parentSpanId: aeb2a416b8796cba + spanId: 9d5a7e824fa7ba3b + startTimeUnixNano: "1706791445912654000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445996222333" + kind: 1 + name: pgstore/getConfigurationIndex + parentSpanId: 5f2c62ec22a5831f + spanId: 5920438aa93a6790 + startTimeUnixNano: "1706791445996221000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445996223000" + kind: 1 + name: pgstore/ConfigurationIndex + parentSpanId: 96ae55c03e5146b3 + spanId: 5f2c62ec22a5831f + startTimeUnixNano: "1706791445996220000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445997209625" + kind: 1 + name: pgstore/acquireAdvisoryLock + parentSpanId: 624db3b2e1cc7f20 + spanId: 70d631526845aa24 + startTimeUnixNano: "1706791445997026000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445998158167" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: e936bad42c7c2620 + spanId: 758070ce974346be + startTimeUnixNano: "1706791445997877000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445998161958" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 624db3b2e1cc7f20 + spanId: e936bad42c7c2620 + startTimeUnixNano: "1706791445997305000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999155375" + kind: 1 + name: pgstore/scanPostgresResource + parentSpanId: f2f75b973ffbece4 + spanId: e87d48de1c7ee8c0 + startTimeUnixNano: "1706791445998945000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999158708" + kind: 1 + name: pgstore/pgResourceInternal + parentSpanId: 494fbd3596fa4bb7 + spanId: f2f75b973ffbece4 + startTimeUnixNano: "1706791445998570000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999159250" + kind: 1 + name: pgstore/pgResource + parentSpanId: 9b1bf4c6a7ee0b64 + spanId: 494fbd3596fa4bb7 + startTimeUnixNano: "1706791445998569000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999195916" + kind: 1 + name: pgstore/pgEditConfiguration + parentSpanId: 624db3b2e1cc7f20 + spanId: 9b1bf4c6a7ee0b64 + startTimeUnixNano: "1706791445998568000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999284667" + kind: 1 + name: pgstore/addTransitiveUpdates + parentSpanId: bd51cf7f70c9e6ec + spanId: e46d93a1301ab208 + startTimeUnixNano: "1706791445999284000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999285208" + kind: 1 + name: pgstore/notify + parentSpanId: 624db3b2e1cc7f20 + spanId: bd51cf7f70c9e6ec + startTimeUnixNano: "1706791445999283000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999453875" + kind: 1 + name: pgstore/releaseAdvisoryLock + parentSpanId: 624db3b2e1cc7f20 + spanId: 92acfcfbc5386ef6 + startTimeUnixNano: "1706791445999286000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999455625" + kind: 1 + name: pgstore/UpdateRollout + parentSpanId: 96ae55c03e5146b3 + spanId: 624db3b2e1cc7f20 + startTimeUnixNano: "1706791445997022000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999455959" + kind: 1 + name: pgstore/UpdateRollouts + parentSpanId: aeb2a416b8796cba + spanId: 96ae55c03e5146b3 + startTimeUnixNano: "1706791445996218000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445999459125" + kind: 1 + name: pgstore/UpdateAllRollouts + parentSpanId: "" + spanId: aeb2a416b8796cba + startTimeUnixNano: "1706791445908223000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - scope: {} + spans: + - attributes: + - key: operation + value: + stringValue: GetConfiguration + endTimeUnixNano: "1706791445376564375" + kind: 1 + name: graphql/GetConfiguration/response + parentSpanId: 723c3f6eb4457b5c + spanId: fd55f461239efdfc + startTimeUnixNano: "1706791445359466000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - attributes: + - key: operation + value: + stringValue: GetConfiguration + endTimeUnixNano: "1706791445376589750" + kind: 1 + name: graphql/GetConfiguration/response + parentSpanId: 3e7909bbebcae0ba + spanId: 01f07757e7bb6612 + startTimeUnixNano: "1706791445359560000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - scope: {} + spans: + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.scheme + value: + stringValue: http + - key: net.host.name + value: + stringValue: bindplane + - key: net.host.port + value: + intValue: "3001" + - key: net.sock.peer.addr + value: + stringValue: 127.0.0.1 + - key: net.sock.peer.port + value: + intValue: "50141" + - key: user_agent.original + value: + stringValue: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.15; rv:123.0) Gecko/20100101 Firefox/123.0 + - key: http.client_ip + value: + stringValue: 127.0.0.1 + - key: http.target + value: + stringValue: /v1/graphql + - key: net.protocol.version + value: + stringValue: "1.1" + - key: http.route + value: + stringValue: /v1/graphql + - key: http.status_code + value: + intValue: "200" + endTimeUnixNano: "1706791445376694750" + kind: 2 + name: /v1/graphql + parentSpanId: "" + spanId: 723c3f6eb4457b5c + startTimeUnixNano: "1706791445332980000" + status: {} + traceId: a3fbd5dc5db5e1734cb54419ca540b66 + - attributes: + - key: http.method + value: + stringValue: POST + - key: http.scheme + value: + stringValue: http + - key: net.host.name + value: + stringValue: bindplane + - key: net.host.port + value: + intValue: "3001" + - key: net.sock.peer.addr + value: + stringValue: 127.0.0.1 + - key: net.sock.peer.port + value: + intValue: "50140" + - key: user_agent.original + value: + stringValue: Mozilla/5.0 (Macintosh; Intel Mac OS X 10.15; rv:123.0) Gecko/20100101 Firefox/123.0 + - key: http.client_ip + value: + stringValue: 127.0.0.1 + - key: http.target + value: + stringValue: /v1/graphql + - key: net.protocol.version + value: + stringValue: "1.1" + - key: http.route + value: + stringValue: /v1/graphql + - key: http.status_code + value: + intValue: "200" + endTimeUnixNano: "1706791445376708291" + kind: 2 + name: /v1/graphql + parentSpanId: "" + spanId: 3e7909bbebcae0ba + startTimeUnixNano: "1706791445332972000" + status: {} + traceId: d70c2b5eea8977bb8a0712f8c2a1fcb4 + - scope: {} + spans: + - endTimeUnixNano: "1706791445913878000" + kind: 1 + name: pgindex/Suggestions + parentSpanId: 9d5a7e824fa7ba3b + spanId: 4c2049c4cd14c987 + startTimeUnixNano: "1706791445912675000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31 + - endTimeUnixNano: "1706791445997017791" + kind: 1 + name: pgindex/Suggestions + parentSpanId: 96ae55c03e5146b3 + spanId: aa69c45bc0970c2f + startTimeUnixNano: "1706791445996229000" + status: {} + traceId: c7f3bb6aa9e7a7dce92d85d1566f2c31