diff --git a/docs/exporters.md b/docs/exporters.md index 17eeb372a..c7a9fe46a 100644 --- a/docs/exporters.md +++ b/docs/exporters.md @@ -14,6 +14,7 @@ Below is a list of supported exporters with links to their documentation pages. | Azure Monitor Exporter | [azuremonitorexporter](https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/v0.91.0/exporter/azuremonitorexporter/README.md) | | Carbon Exporter | [carbonexporter](https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/v0.91.0/exporter/carbonexporter/README.md) | | Chronicle Exporter | [chronicleexporter](../exporter/chronicleexporter/README.md) | +| Chronicle Forwarder Exporter | [chronicleexporter](../exporter/chronicleforwarderexporter/README.md) | | ClickHouse Exporter | [clickhouseexporter](https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/v0.91.0/exporter/clickhouseexporter/README.md) | | Coralogix Exporter | [coralogixexporter](https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/v0.91.0/exporter/coralogixexporter/README.md) | | Datadog Exporter | [datadogexporter](https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/v0.91.0/exporter/datadogexporter/README.md) | diff --git a/exporter/chronicleforwarderexporter/README.md b/exporter/chronicleforwarderexporter/README.md new file mode 100644 index 000000000..534a76178 --- /dev/null +++ b/exporter/chronicleforwarderexporter/README.md @@ -0,0 +1,52 @@ +# Chronicle Forwarder Exporter + +The Chronicle Forwarder Exporter is designed for forwarding logs to a Chronicle Forwarder endpoint using either Syslog or File-based methods. This exporter supports customization of data export types and various configuration options to tailor the connection and data handling to specific needs. + +## Minimum Agent Versions + +- Introduced: [v1.42.0](https://github.com/observIQ/bindplane-agent/releases/tag/v1.42.0) + +## Supported Pipelines + +- Logs + +## How It Works + +1. For Syslog, it establishes a network connection to the specified Chronicle forwarder endpoint. +2. For File, it writes logs to a specified file path. + +## Configuration + +| Field | Type | Default Value | Required | Description | +| -------------------- | ------ | ----------------- | -------- | ------------------------------------------------- | +| export_type | string | `syslog` | `true` | Type of export, either `syslog` or `file`. | +| raw_log_field | string | | `false` | The field name to send raw logs to Chronicle. | +| syslog.endpoint | string | `127.0.0.1:10514` | `false` | The Chronicle forwarder endpoint. | +| syslog.network | string | `tcp` | `false` | The network protocol to use (e.g., `tcp`, `udp`). | +| syslog.tls.key_file | string | | `false` | Configure the receiver to use TLS. | +| syslog.tls.cert_file | string | | `false` | Configure the receiver to use TLS. | +| file.path | string | | `false` | The path to the file for storing logs. | + +## Example Configurations + +### Syslog Configuration Example + +```yaml +chronicleforwarder: + export_type: "syslog" + syslog: + host: "syslog.example.com" + port: 10514 + network: "tcp" +``` + +### File Configuration Example + +```yaml +chronicleforwarder: + export_type: "file" + file: + path: "/path/to/logfile" +``` + +--- diff --git a/exporter/chronicleforwarderexporter/config.go b/exporter/chronicleforwarderexporter/config.go new file mode 100644 index 000000000..29fe15e30 --- /dev/null +++ b/exporter/chronicleforwarderexporter/config.go @@ -0,0 +1,113 @@ +// 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 chronicleforwarderexporter + +import ( + "errors" + "fmt" + + "github.com/observiq/bindplane-agent/expr" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/config/confignet" + "go.opentelemetry.io/collector/config/configtls" + "go.opentelemetry.io/collector/exporter/exporterhelper" + "go.uber.org/zap" +) + +const ( + // exportTypeSyslog is the syslog export type. + exportTypeSyslog = "syslog" + + // exportTypeFile is the file export type. + exportTypeFile = "file" +) + +// Config defines configuration for the Chronicle exporter. +type Config struct { + exporterhelper.TimeoutSettings `mapstructure:",squash"` + exporterhelper.QueueSettings `mapstructure:"sending_queue"` + exporterhelper.RetrySettings `mapstructure:"retry_on_failure"` + + // ExportType is the type of export to use. + ExportType string `mapstructure:"export_type"` + + // Syslog is the configuration for the connection to the Chronicle forwarder. + Syslog SyslogConfig `mapstructure:"syslog"` + + // File is the configuration for the connection to the Chronicle forwarder. + File File `mapstructure:"file"` + + // RawLogField is the field name that will be used to send raw logs to Chronicle. + RawLogField string `mapstructure:"raw_log_field"` +} + +// SyslogConfig defines configuration for the Chronicle forwarder connection. +type SyslogConfig struct { + confignet.NetAddr `mapstructure:",squash"` + + // TLSSetting struct exposes TLS client configuration. + TLSSetting *configtls.TLSClientSetting `mapstructure:"tls"` +} + +// File defines configuration for sending to. +type File struct { + // Path is the path to the file to send to Chronicle. + Path string `mapstructure:"path"` +} + +// validate validates the Syslog configuration. +func (s *SyslogConfig) validate() error { + if s.NetAddr.Endpoint == "" { + return errors.New("incomplete syslog configuration: endpoint is required") + } + return nil +} + +// validate validates the File configuration. +func (f *File) validate() error { + if f.Path == "" { + return errors.New("file path is required for file export type") + } + return nil +} + +// Validate validates the Chronicle exporter configuration. +func (cfg *Config) Validate() error { + if cfg.ExportType != exportTypeSyslog && cfg.ExportType != exportTypeFile { + return errors.New("export_type must be either 'syslog' or 'file'") + } + + if cfg.ExportType == exportTypeSyslog { + if err := cfg.Syslog.validate(); err != nil { + return err + } + } + + if cfg.ExportType == exportTypeFile { + if err := cfg.File.validate(); err != nil { + return err + } + } + + if cfg.RawLogField != "" { + _, err := expr.NewOTTLLogRecordExpression(cfg.RawLogField, component.TelemetrySettings{ + Logger: zap.NewNop(), + }) + if err != nil { + return fmt.Errorf("raw_log_field is invalid: %s", err) + } + } + return nil +} diff --git a/exporter/chronicleforwarderexporter/config_test.go b/exporter/chronicleforwarderexporter/config_test.go new file mode 100644 index 000000000..6fb5f809f --- /dev/null +++ b/exporter/chronicleforwarderexporter/config_test.go @@ -0,0 +1,86 @@ +// 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 chronicleforwarderexporter + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/config/confignet" +) + +func TestConfig_Validate(t *testing.T) { + tests := []struct { + name string + cfg Config + wantErr bool + }{ + { + name: "Valid syslog config", + cfg: Config{ + ExportType: exportTypeSyslog, + Syslog: SyslogConfig{ + NetAddr: confignet.NetAddr{ + Endpoint: "localhost:514", + Transport: "tcp", + }, + }, + }, + wantErr: false, + }, + { + name: "Invalid syslog config - missing host", + cfg: Config{ + ExportType: exportTypeSyslog, + Syslog: SyslogConfig{ + NetAddr: confignet.NetAddr{ + Endpoint: "", + Transport: "tcp", + }, + }, + }, + wantErr: true, + }, + { + name: "Valid file config", + cfg: Config{ + ExportType: exportTypeFile, + File: File{ + Path: "/path/to/file", + }, + }, + wantErr: false, + }, + { + name: "Invalid file config - missing path", + cfg: Config{ + ExportType: exportTypeFile, + File: File{}, + }, + wantErr: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + err := tt.cfg.Validate() + if tt.wantErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} diff --git a/exporter/chronicleforwarderexporter/doc.go b/exporter/chronicleforwarderexporter/doc.go new file mode 100644 index 000000000..62c7a1040 --- /dev/null +++ b/exporter/chronicleforwarderexporter/doc.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. + +//go:generate mdatagen metadata.yaml + +// Package chronicleforwarderexporter exports OpenTelemetry data to an endpoint or file. +package chronicleforwarderexporter // import "github.com/observiq/bindplane-agent/exporter/azureblobexporter" diff --git a/exporter/chronicleforwarderexporter/exporter.go b/exporter/chronicleforwarderexporter/exporter.go new file mode 100644 index 000000000..16449d8ac --- /dev/null +++ b/exporter/chronicleforwarderexporter/exporter.go @@ -0,0 +1,115 @@ +// 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 chronicleforwarderexporter + +import ( + "context" + "crypto/tls" + "errors" + "fmt" + "io" + "net" + "os" + "strings" + + "go.opentelemetry.io/collector/consumer" + "go.opentelemetry.io/collector/exporter" + "go.opentelemetry.io/collector/pdata/plog" + "go.uber.org/zap" +) + +type chronicleForwarderExporter struct { + cfg *Config + logger *zap.Logger + marshaler logMarshaler + endpoint string +} + +func newExporter(cfg *Config, params exporter.CreateSettings) (*chronicleForwarderExporter, error) { + return &chronicleForwarderExporter{ + cfg: cfg, + logger: params.Logger, + marshaler: newMarshaler(*cfg, params.TelemetrySettings), + }, nil +} + +func (ce *chronicleForwarderExporter) Capabilities() consumer.Capabilities { + return consumer.Capabilities{MutatesData: false} +} + +func (ce *chronicleForwarderExporter) logsDataPusher(ctx context.Context, ld plog.Logs) error { + // Open connection or file before sending each payload + writer, err := ce.openWriter() + if err != nil { + return fmt.Errorf("open writer: %w", err) + } + defer writer.Close() + + payloads, err := ce.marshaler.MarshalRawLogs(ctx, ld) + if err != nil { + return fmt.Errorf("marshal logs: %w", err) + } + + for _, payload := range payloads { + if err := ce.send(payload, writer); err != nil { + return fmt.Errorf("upload to Chronicle forwarder: %w", err) + } + } + + return nil +} + +func (ce *chronicleForwarderExporter) openWriter() (io.WriteCloser, error) { + switch ce.cfg.ExportType { + case exportTypeSyslog: + return ce.openSyslogWriter() + case exportTypeFile: + return ce.openFileWriter() + default: + return nil, errors.New("unsupported export type") + } +} + +func (ce *chronicleForwarderExporter) openFileWriter() (io.WriteCloser, error) { + return os.OpenFile(ce.cfg.File.Path, os.O_APPEND|os.O_CREATE|os.O_WRONLY, 0600) +} + +func (ce *chronicleForwarderExporter) openSyslogWriter() (io.WriteCloser, error) { + var conn net.Conn + var err error + if ce.cfg.Syslog.TLSSetting != nil { + tlsConfig, err := ce.cfg.Syslog.TLSSetting.LoadTLSConfig() + if err != nil { + return nil, fmt.Errorf("load TLS config: %w", err) + } + conn, err = tls.Dial(ce.cfg.Syslog.NetAddr.Transport, ce.cfg.Syslog.NetAddr.Endpoint, tlsConfig) + } else { + conn, err = net.Dial(ce.cfg.Syslog.NetAddr.Transport, ce.cfg.Syslog.NetAddr.Endpoint) + } + + if err != nil { + return nil, fmt.Errorf("dial: %w", err) + } + return conn, nil +} + +func (ce *chronicleForwarderExporter) send(msg string, writer io.WriteCloser) error { + if !strings.HasSuffix(msg, "\n") { + msg = fmt.Sprintf("%s%s", msg, "\n") + } + + _, err := io.WriteString(writer, msg) + return err +} diff --git a/exporter/chronicleforwarderexporter/exporter_test.go b/exporter/chronicleforwarderexporter/exporter_test.go new file mode 100644 index 000000000..5cc3372da --- /dev/null +++ b/exporter/chronicleforwarderexporter/exporter_test.go @@ -0,0 +1,138 @@ +// 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 chronicleforwarderexporter + +import ( + "context" + "log" + "net" + "os" + "testing" + "time" + + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/config/confignet" + "go.opentelemetry.io/collector/exporter" +) + +func Test_exporter_Capabilities(t *testing.T) { + exp := &chronicleForwarderExporter{} + capabilities := exp.Capabilities() + require.False(t, capabilities.MutatesData) +} + +func TestLogDataPushingFile(t *testing.T) { + // Open a temporary file for testing + f, err := os.CreateTemp("", "test") + require.NoError(t, err) + defer f.Close() + defer os.Remove(f.Name()) // Clean up the file afterwards + + cfg := &Config{ + ExportType: exportTypeFile, + File: File{ + Path: f.Name(), + }, + } + exporter, _ := newExporter(cfg, exporter.CreateSettings{}) + + // Mock log data + ld := mockLogs(mockLogRecord(t, "test", map[string]any{"test": "test"})) + + err = exporter.logsDataPusher(context.Background(), ld) + require.NoError(t, err) + + // Read the contents of the file + content, err := os.ReadFile(f.Name()) + require.NoError(t, err) + + // Convert the content to a string and compare with the expected output + receivedData := string(content) + expectedData := "{\"attributes\":{\"test\":\"test\"},\"body\":\"test\",\"resource_attributes\":{}}\n" + require.Equal(t, expectedData, receivedData, "File content does not match expected output") +} + +func TestLogDataPushingNetwork(t *testing.T) { + // Channel to signal when log is received + logReceived := make(chan bool) + + // Set up a mock Syslog server + ln, err := net.Listen("tcp", "localhost:0") + require.NoError(t, err) + defer ln.Close() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + for { + select { + case <-ctx.Done(): + return + default: + conn, err := ln.Accept() + if err != nil { + log.Println("Error accepting connection:", err) + return + } + go handleSyslogConnection(t, conn, logReceived) + } + } + }() + + // Configure the exporter to use the mock Syslog server + cfg := &Config{ + ExportType: exportTypeSyslog, + Syslog: SyslogConfig{ + NetAddr: confignet.NetAddr{ + Endpoint: ln.Addr().String(), + Transport: "tcp", + }, + }, + } + exporter, _ := newExporter(cfg, exporter.CreateSettings{}) + + // Mock log data + ld := mockLogs(mockLogRecord(t, "test", map[string]any{"test": "test"})) + + // Test log data pushing + err = exporter.logsDataPusher(context.Background(), ld) + require.NoError(t, err) + + select { + case <-logReceived: + case <-time.After(5 * time.Second): + t.Fatal("Timeout waiting for log to be received") + } +} + +func handleSyslogConnection(t *testing.T, conn net.Conn, logReceived chan bool) { + defer conn.Close() + + // Buffer to store the received data + buf := make([]byte, 1024) + + // Read data from the connection + n, err := conn.Read(buf) + require.NoError(t, err) + + // Extract the received message + receivedData := string(buf[:n]) + + require.Equal(t, "{\"attributes\":{\"test\":\"test\"},\"body\":\"test\",\"resource_attributes\":{}}\n", receivedData) + + logReceived <- true + conn.Close() +} diff --git a/exporter/chronicleforwarderexporter/factory.go b/exporter/chronicleforwarderexporter/factory.go new file mode 100644 index 000000000..679c80214 --- /dev/null +++ b/exporter/chronicleforwarderexporter/factory.go @@ -0,0 +1,78 @@ +// 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 chronicleforwarderexporter + +import ( + "context" + "errors" + + "github.com/observiq/bindplane-agent/exporter/chronicleforwarderexporter/internal/metadata" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/config/confignet" + "go.opentelemetry.io/collector/exporter" + "go.opentelemetry.io/collector/exporter/exporterhelper" +) + +// NewFactory creates a new Chronicle exporter factory. +func NewFactory() exporter.Factory { + return exporter.NewFactory( + metadata.Type, + createDefaultConfig, + exporter.WithLogs(createLogsExporter, metadata.LogsStability)) +} + +// createDefaultConfig creates the default configuration for the exporter. +func createDefaultConfig() component.Config { + return &Config{ + TimeoutSettings: exporterhelper.NewDefaultTimeoutSettings(), + QueueSettings: exporterhelper.NewDefaultQueueSettings(), + RetrySettings: exporterhelper.NewDefaultRetrySettings(), + ExportType: exportTypeSyslog, + Syslog: SyslogConfig{ + NetAddr: confignet.NetAddr{ + Endpoint: "127.0.0.1:10514", + Transport: "tcp", + }, + }, + } +} + +// createLogsExporter creates a new log exporter based on this config. +func createLogsExporter( + ctx context.Context, + params exporter.CreateSettings, + cfg component.Config, +) (exporter.Logs, error) { + forwarderCfg, ok := cfg.(*Config) + if !ok { + return nil, errors.New("invalid config type") + } + + exp, err := newExporter(forwarderCfg, params) + if err != nil { + return nil, err + } + + return exporterhelper.NewLogsExporter( + ctx, + params, + forwarderCfg, + exp.logsDataPusher, + exporterhelper.WithCapabilities(exp.Capabilities()), + exporterhelper.WithTimeout(forwarderCfg.TimeoutSettings), + exporterhelper.WithQueue(forwarderCfg.QueueSettings), + exporterhelper.WithRetry(forwarderCfg.RetrySettings), + ) +} diff --git a/exporter/chronicleforwarderexporter/factory_test.go b/exporter/chronicleforwarderexporter/factory_test.go new file mode 100644 index 000000000..7a9fa4c76 --- /dev/null +++ b/exporter/chronicleforwarderexporter/factory_test.go @@ -0,0 +1,41 @@ +// 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 chronicleforwarderexporter // import "github.com/observiq/bindplane-agent/exporter/azureblobexporter" + +import ( + "testing" + + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/config/confignet" + "go.opentelemetry.io/collector/exporter/exporterhelper" +) + +func Test_createDefaultConfig(t *testing.T) { + expectedCfg := &Config{ + TimeoutSettings: exporterhelper.NewDefaultTimeoutSettings(), + QueueSettings: exporterhelper.NewDefaultQueueSettings(), + RetrySettings: exporterhelper.NewDefaultRetrySettings(), + ExportType: exportTypeSyslog, + Syslog: SyslogConfig{ + NetAddr: confignet.NetAddr{ + Endpoint: "127.0.0.1:10514", + Transport: "tcp", + }, + }, + } + + actual := createDefaultConfig() + require.Equal(t, expectedCfg, actual) +} diff --git a/exporter/chronicleforwarderexporter/go.mod b/exporter/chronicleforwarderexporter/go.mod new file mode 100644 index 000000000..39ad24a6f --- /dev/null +++ b/exporter/chronicleforwarderexporter/go.mod @@ -0,0 +1,65 @@ +module github.com/observiq/bindplane-agent/exporter/chronicleforwarderexporter + +go 1.20 + +require ( + github.com/observiq/bindplane-agent/expr v1.41.0 + github.com/open-telemetry/opentelemetry-collector-contrib/pkg/ottl v0.91.0 + github.com/stretchr/testify v1.8.4 + go.opentelemetry.io/collector/component v0.91.0 + go.opentelemetry.io/collector/config/configtls v0.91.0 + go.opentelemetry.io/collector/consumer v0.91.0 + go.opentelemetry.io/collector/exporter v0.91.0 + go.opentelemetry.io/collector/pdata v1.0.0 + go.uber.org/zap v1.26.0 +) + +require ( + github.com/fsnotify/fsnotify v1.7.0 // indirect + go.opentelemetry.io/collector/config/configopaque v0.91.0 // indirect +) + +require ( + github.com/alecthomas/participle/v2 v2.1.1 // indirect + github.com/antonmedv/expr v1.15.5 // indirect + github.com/cenkalti/backoff/v4 v4.2.1 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/gobwas/glob v0.2.3 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/protobuf v1.5.3 // indirect + github.com/google/uuid v1.4.0 // indirect + github.com/hashicorp/go-version v1.6.0 // indirect + github.com/iancoleman/strcase v0.3.0 // indirect + github.com/json-iterator/go v1.1.12 // indirect + github.com/knadh/koanf/maps v0.1.1 // indirect + github.com/knadh/koanf/providers/confmap v0.1.0 // indirect + github.com/knadh/koanf/v2 v2.0.1 // indirect + github.com/mitchellh/copystructure v1.2.0 // indirect + github.com/mitchellh/mapstructure v1.5.1-0.20220423185008-bf980b35cac4 // indirect + github.com/mitchellh/reflectwalk v1.0.2 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect + github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal v0.91.0 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + go.opencensus.io v0.24.0 // indirect + go.opentelemetry.io/collector v0.91.0 // indirect + go.opentelemetry.io/collector/config/confignet v0.91.0 + go.opentelemetry.io/collector/config/configtelemetry v0.91.0 // indirect + go.opentelemetry.io/collector/confmap v0.91.0 // indirect + go.opentelemetry.io/collector/extension v0.91.0 // indirect + go.opentelemetry.io/collector/featuregate v1.0.0 // indirect + go.opentelemetry.io/otel v1.21.0 // indirect + go.opentelemetry.io/otel/metric v1.21.0 // indirect + go.opentelemetry.io/otel/trace v1.21.0 // indirect + go.uber.org/multierr v1.11.0 // indirect + golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect + golang.org/x/net v0.18.0 // indirect + golang.org/x/sys v0.15.0 // indirect + golang.org/x/text v0.14.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d // indirect + google.golang.org/grpc v1.59.0 // indirect + google.golang.org/protobuf v1.31.0 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect +) + +replace github.com/observiq/bindplane-agent/expr => ../../expr diff --git a/exporter/chronicleforwarderexporter/go.sum b/exporter/chronicleforwarderexporter/go.sum new file mode 100644 index 000000000..f37dab312 --- /dev/null +++ b/exporter/chronicleforwarderexporter/go.sum @@ -0,0 +1,240 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +contrib.go.opencensus.io/exporter/prometheus v0.4.2 h1:sqfsYl5GIY/L570iT+l93ehxaWJs2/OwXtiWwew3oAg= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/alecthomas/assert/v2 v2.3.0 h1:mAsH2wmvjsuvyBvAmCtm7zFsBlb8mIHx5ySLVdDZXL0= +github.com/alecthomas/participle/v2 v2.1.1 h1:hrjKESvSqGHzRb4yW1ciisFJ4p3MGYih6icjJvbsmV8= +github.com/alecthomas/participle/v2 v2.1.1/go.mod h1:Y1+hAs8DHPmc3YUFzqllV+eSQ9ljPTk0ZkPMtEdAx2c= +github.com/alecthomas/repr v0.2.0 h1:HAzS41CIzNW5syS8Mf9UwXhNH1J9aix/BvDRf1Ml2Yk= +github.com/antonmedv/expr v1.15.5 h1:y0Iz3cEwmpRz5/r3w4qQR0MfIqJGdGM1zbhD/v0G5Vg= +github.com/antonmedv/expr v1.15.5/go.mod h1:0E/6TxnOlRNp81GMzX9QfDPAmHo2Phg00y4JUv1ihsE= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= +github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.1-0.20191026205805-5f8ba28d4473/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA= +github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM= +github.com/go-kit/log v0.2.1 h1:MRVx0/zhvdseW+Gza6N9rVzU/IVzaeE1SFI4raAhmBU= +github.com/go-logfmt/logfmt v0.5.1 h1:otpy5pqBCBZ1ng9RQ0dPu4PN7ba75Y/aA+UpowDyNVA= +github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/gobwas/glob v0.2.3 h1:A4xDbljILXROh+kObIiy5kIaPYD8e96x1tgBhUI5J+Y= +github.com/gobwas/glob v0.2.3/go.mod h1:d3Ez4x06l9bZtSvzIay5+Yzi0fmZzPgnTbPcKjJAkT8= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.1/go.mod h1:U8fpvMrcmy5pZrNK1lt4xCsGvpyWQ/VVv6QDs8UjoX8= +github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= +github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.3/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= +github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.4.0 h1:MtMxsa51/r9yyhkyLsVeVt0B+BGQZzpQiTQ4eHZ8bc4= +github.com/google/uuid v1.4.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/hashicorp/go-version v1.6.0 h1:feTTfFNnjP967rlCxM/I9g701jU+RN74YKx2mOkIeek= +github.com/hashicorp/go-version v1.6.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hexops/gotextdiff v1.0.3 h1:gitA9+qJrrTCsiCl7+kh75nPqQt1cx4ZkudSTLoUqJM= +github.com/iancoleman/strcase v0.3.0 h1:nTXanmYxhfFAMjZL34Ov6gkzEsSJZ5DbhxWjvSASxEI= +github.com/iancoleman/strcase v0.3.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho= +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/knadh/koanf/maps v0.1.1 h1:G5TjmUh2D7G2YWf5SQQqSiHRJEjaicvU0KpypqB3NIs= +github.com/knadh/koanf/maps v0.1.1/go.mod h1:npD/QZY3V6ghQDdcQzl1W4ICNVTkohC8E73eI2xW4yI= +github.com/knadh/koanf/providers/confmap v0.1.0 h1:gOkxhHkemwG4LezxxN8DMOFopOPghxRVp7JbIvdvqzU= +github.com/knadh/koanf/providers/confmap v0.1.0/go.mod h1:2uLhxQzJnyHKfxG927awZC7+fyHFdQkd697K4MdLnIU= +github.com/knadh/koanf/v2 v2.0.1 h1:1dYGITt1I23x8cfx8ZnldtezdyaZtfAuRtIFOiRzK7g= +github.com/knadh/koanf/v2 v2.0.1/go.mod h1:ZeiIlIDXTE7w1lMT6UVcNiRAS2/rCeLn/GdLNvY1Dus= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= +github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= +github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= +github.com/mitchellh/mapstructure v1.5.1-0.20220423185008-bf980b35cac4 h1:BpfhmLKZf+SjVanKKhCgf3bg+511DmU9eDQTen7LLbY= +github.com/mitchellh/mapstructure v1.5.1-0.20220423185008-bf980b35cac4/go.mod h1:bFUtVrKA4DC2yAKiSyO/QUcy7e+RRV2QTWOzhPopBRo= +github.com/mitchellh/reflectwalk v1.0.2 h1:G2LzWKi524PWgd3mLHV8Y5k7s6XUvT0Gef6zxSIeXaQ= +github.com/mitchellh/reflectwalk v1.0.2/go.mod h1:mSTlrgnPZtwu0c4WaC2kGObEpuNDbx0jmZXqmk4esnw= +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/internal/coreinternal v0.91.0 h1:I3MFZXcQdnATObbeKseHLEWOWMFt1jHhHCbeunBw3mE= +github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal v0.91.0/go.mod h1:xHPYTciFeEEE2HnPu65FMgsCQFYNns66mqiHsMqb+HM= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/ottl v0.91.0 h1:H2XRo5joSzcBhAvOrch7/p+MHighMshJpBdOWji0qh4= +github.com/open-telemetry/opentelemetry-collector-contrib/pkg/ottl v0.91.0/go.mod h1:+5u+yVQRH/9RmqWwKKLtmGvbopeq6uxRCZDYO7PI7tE= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v1.17.0 h1:rl2sfwZMtSthVU752MqfjQozy7blglC+1SOtjMAMh+Q= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.5.0 h1:VQw1hfvPvk3Uv6Qf29VrPF32JB6rtbgI6cYPYQjL0Qw= +github.com/prometheus/common v0.45.0 h1:2BGz0eBc2hdMDLnO/8n0jeB3oPrt2D08CekT0lneoxM= +github.com/prometheus/procfs v0.11.1 h1:xRC8Iq1yyca5ypa9n1EZnWZkt7dwcoRPQwX/5gwaUuI= +github.com/prometheus/statsd_exporter v0.22.7 h1:7Pji/i2GuhK6Lu7DHrtTkFmNBCudCPT1pX2CziuyQR0= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= +github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= +github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= +go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= +go.opentelemetry.io/collector v0.91.0 h1:C7sGUJDJ5nwm+CkWpAaVP3lNsuYpwSRbkmLncFjkmO8= +go.opentelemetry.io/collector v0.91.0/go.mod h1:YhQpIDZsn+bICAAqgBwXk9wqK8GKZDv+aogfG52zUuE= +go.opentelemetry.io/collector/component v0.91.0 h1:aBT1i2zGyfh9PalYJLfXVvQp+osHyalwyDFselI1CtA= +go.opentelemetry.io/collector/component v0.91.0/go.mod h1:2KBHvjNFdU7oOjsObQeC4Ta2Ef607OISU5obznW00fw= +go.opentelemetry.io/collector/config/confignet v0.91.0 h1:3huNXh04O3wXaN4qPhmmiefyz4dYbOlNcR/OKMByqig= +go.opentelemetry.io/collector/config/confignet v0.91.0/go.mod h1:cpO8JYWGONaViOygKVw+Hd2UoBcn2cUiyi0WWeFTwJY= +go.opentelemetry.io/collector/config/configopaque v0.91.0 h1:bQgJPyARbuXAsU2p6h2YbEm1kHb1stS6hg42ekyMZmI= +go.opentelemetry.io/collector/config/configopaque v0.91.0/go.mod h1:TPCHaU+QXiEV+JXbgyr6mSErTI9chwQyasDVMdJr3eY= +go.opentelemetry.io/collector/config/configtelemetry v0.91.0 h1:mEwvqrYfwUJ7LwYfpcF9M8z7LHFoYaKhEPhnERD/88E= +go.opentelemetry.io/collector/config/configtelemetry v0.91.0/go.mod h1:+LAXM5WFMW/UbTlAuSs6L/W72WC+q8TBJt/6z39FPOU= +go.opentelemetry.io/collector/config/configtls v0.91.0 h1:lZromNeOslPwyVlTPMOzF2q++SY+VONvfH3cDqA0kKk= +go.opentelemetry.io/collector/config/configtls v0.91.0/go.mod h1:E+CW5gZoH8V3z5aSlZxwiof7GAcayzn1HRM+uRILLEI= +go.opentelemetry.io/collector/confmap v0.91.0 h1:7U2MT+u74oEzq/WWrpXSLKB7nX5jPNC4drwtQdYfwKk= +go.opentelemetry.io/collector/confmap v0.91.0/go.mod h1:uxV+fZ85kG31oovL6Cl3fAMQ3RRPwUvfAbbA9WT1Yhk= +go.opentelemetry.io/collector/consumer v0.91.0 h1:0nU1lUe2S0b8iOmF3w3R/9Dt24n413thRTbXz/nJgrM= +go.opentelemetry.io/collector/consumer v0.91.0/go.mod h1:phTUQmr7hpYfwXyDXo4mFHVjYrlSbZE+nZYlKlbVxGs= +go.opentelemetry.io/collector/exporter v0.91.0 h1:guWcGflFjaenp3BMxAmAKjb8RQG80jQQKjuUFouS+z8= +go.opentelemetry.io/collector/exporter v0.91.0/go.mod h1:hkOBunNNWu6CaTtkRsCJ/OJ509REJZg+DDElevFIQCQ= +go.opentelemetry.io/collector/extension v0.91.0 h1:bkoSLgnWm4g6n+RLmyKG6Up7dr8KmJy68quonoLZnr0= +go.opentelemetry.io/collector/extension v0.91.0/go.mod h1:F3r0fVTTh4sYR0GVv51Qez8lk8v77kTDPdyMOp6A2kg= +go.opentelemetry.io/collector/featuregate v1.0.0 h1:5MGqe2v5zxaoo73BUOvUTunftX5J8RGrbFsC2Ha7N3g= +go.opentelemetry.io/collector/featuregate v1.0.0/go.mod h1:xGbRuw+GbutRtVVSEy3YR2yuOlEyiUMhN2M9DJljgqY= +go.opentelemetry.io/collector/pdata v1.0.0 h1:ECP2jnLztewsHmL1opL8BeMtWVc7/oSlKNhfY9jP8ec= +go.opentelemetry.io/collector/pdata v1.0.0/go.mod h1:TsDFgs4JLNG7t6x9D8kGswXUz4mme+MyNChHx8zSF6k= +go.opentelemetry.io/collector/receiver v0.91.0 h1:0TZF/0OXoJtxgm+mvOinRRXo9LgVyOsOgCQfWkNGXJA= +go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= +go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= +go.opentelemetry.io/otel/exporters/prometheus v0.44.1-0.20231201153405-6027c1ae76f2 h1:TnhkxGJ5qPHAMIMI4r+HPT/BbpoHxqn4xONJrok054o= +go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= +go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= +go.opentelemetry.io/otel/sdk/metric v1.21.0 h1:smhI5oD714d6jHE6Tie36fPx4WDFIg+Y6RfAY4ICcR0= +go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= +go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= +go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= +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.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= +go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +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/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20231127185646-65229373498e h1:Gvh4YaCaXNs6dKTlfgismwWZKyjVZXwOPfIyUaqU3No= +golang.org/x/exp v0.0.0-20231127185646-65229373498e/go.mod h1:iRJReGqOEeBhDZGkGbynYwcHlctCvnjTYIamk7uXpHI= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +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-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +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.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.18.0 h1:mIYleuAkSbHh0tCv7RvjL3F6ZVbLjq4+R7zbOn3Kokg= +golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +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-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= +golang.org/x/sys v0.15.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.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ= +golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +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/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d h1:uvYuEyMHKNt+lT4K3bN6fGswmK8qSvcreM3BwjDh+y4= +google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go.mod h1:+Bk1OCOj40wS2hwAMA+aCW9ypzm63QTBBHp6lQ3p+9M= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= +google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= +google.golang.org/grpc v1.59.0 h1:Z5Iec2pjwb+LEOqzpB2MR12/eKFhDPhuqW91O+4bwUk= +google.golang.org/grpc v1.59.0/go.mod h1:aUPDwccQo6OTjy7Hct4AfBPD1GptF4fyUjIkQ9YtF98= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.22.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= +google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= +google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= +google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= +google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +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/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= diff --git a/exporter/chronicleforwarderexporter/internal/metadata/generated_status.go b/exporter/chronicleforwarderexporter/internal/metadata/generated_status.go new file mode 100644 index 000000000..be6e90bb3 --- /dev/null +++ b/exporter/chronicleforwarderexporter/internal/metadata/generated_status.go @@ -0,0 +1,12 @@ +// Code generated by mdatagen. DO NOT EDIT. + +package metadata + +import ( + "go.opentelemetry.io/collector/component" +) + +const ( + Type = "chronicleforwarder" + LogsStability = component.StabilityLevelAlpha +) diff --git a/exporter/chronicleforwarderexporter/marshal.go b/exporter/chronicleforwarderexporter/marshal.go new file mode 100644 index 000000000..3314ad754 --- /dev/null +++ b/exporter/chronicleforwarderexporter/marshal.go @@ -0,0 +1,122 @@ +// 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 chronicleforwarderexporter + +import ( + "context" + "encoding/json" + "fmt" + + "github.com/observiq/bindplane-agent/expr" + "github.com/open-telemetry/opentelemetry-collector-contrib/pkg/ottl/contexts/ottllog" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" + "go.uber.org/zap" +) + +const logTypeField = `attributes["log_type"]` + +type marshaler struct { + cfg Config + teleSettings component.TelemetrySettings +} + +type logMarshaler interface { + MarshalRawLogs(ctx context.Context, ld plog.Logs) ([]string, error) +} + +func newMarshaler(cfg Config, teleSettings component.TelemetrySettings) *marshaler { + return &marshaler{ + cfg: cfg, + teleSettings: teleSettings, + } +} + +func (m *marshaler) MarshalRawLogs(ctx context.Context, ld plog.Logs) ([]string, error) { + var rawLogs []string + + for i := 0; i < ld.ResourceLogs().Len(); i++ { + resourceLog := ld.ResourceLogs().At(i) + for j := 0; j < resourceLog.ScopeLogs().Len(); j++ { + scopeLog := resourceLog.ScopeLogs().At(j) + for k := 0; k < scopeLog.LogRecords().Len(); k++ { + logRecord := scopeLog.LogRecords().At(k) + rawLog, err := m.getRawLog(ctx, logRecord, scopeLog.Scope(), resourceLog.Resource()) + if err != nil { + m.teleSettings.Logger.Error("Error processing log record", zap.Error(err)) + continue + } + + if rawLog == "" { + m.teleSettings.Logger.Error("Error processing log record: raw log is empty") + continue + } + + rawLogs = append(rawLogs, rawLog) + } + } + } + + return rawLogs, nil +} + +func (m *marshaler) getRawLog(ctx context.Context, logRecord plog.LogRecord, scope pcommon.InstrumentationScope, resource pcommon.Resource) (string, error) { + if m.cfg.RawLogField == "" { + entireLogRecord := map[string]any{ + "body": logRecord.Body().AsRaw(), + "attributes": logRecord.Attributes().AsRaw(), + "resource_attributes": resource.Attributes().AsRaw(), + } + + bytesLogRecord, err := json.Marshal(entireLogRecord) + if err != nil { + return "", fmt.Errorf("marshal log record: %w", err) + } + + return string(bytesLogRecord), nil + } + return m.getRawField(ctx, m.cfg.RawLogField, logRecord, scope, resource) +} + +func (m *marshaler) getRawField(ctx context.Context, field string, logRecord plog.LogRecord, scope pcommon.InstrumentationScope, resource pcommon.Resource) (string, error) { + lrExpr, err := expr.NewOTTLLogRecordExpression(field, m.teleSettings) + if err != nil { + return "", fmt.Errorf("raw_log_field is invalid: %s", err) + } + tCtx := ottllog.NewTransformContext(logRecord, scope, resource) + + lrExprResult, err := lrExpr.Execute(ctx, tCtx) + if err != nil { + return "", fmt.Errorf("execute log record expression: %w", err) + } + + if lrExprResult == nil { + return "", nil + } + + switch result := lrExprResult.(type) { + case string: + return result, nil + case pcommon.Map: + bytes, err := json.Marshal(result.AsRaw()) + if err != nil { + return "", fmt.Errorf("marshal log record expression result: %w", err) + } + return string(bytes), nil + default: + return "", fmt.Errorf("unsupported log record expression result type: %T", lrExprResult) + } +} diff --git a/exporter/chronicleforwarderexporter/marshal_test.go b/exporter/chronicleforwarderexporter/marshal_test.go new file mode 100644 index 000000000..fc5d99d5e --- /dev/null +++ b/exporter/chronicleforwarderexporter/marshal_test.go @@ -0,0 +1,181 @@ +// 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 chronicleforwarderexporter + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" + "go.opentelemetry.io/collector/component" + "go.opentelemetry.io/collector/pdata/pcommon" + "go.opentelemetry.io/collector/pdata/plog" + "go.uber.org/zap" + "go.uber.org/zap/zaptest/observer" +) + +var testTime = time.Date(2023, 1, 2, 3, 4, 5, 6, time.UTC) + +// mockLogRecord creates a simple mock plog.LogRecord for testing. +func mockLogRecord(t *testing.T, body string, attributes map[string]any) plog.LogRecord { + lr := plog.NewLogRecord() + lr.Body().SetStr(body) + lr.Attributes().EnsureCapacity(len(attributes)) + lr.SetTimestamp(pcommon.NewTimestampFromTime(testTime)) + for k, v := range attributes { + switch v.(type) { + case string: + lr.Attributes().PutStr(k, v.(string)) + case map[string]any: + lr.Attributes().FromRaw(attributes) + case int: + lr.Attributes().PutInt(k, int64(v.(int))) + default: + t.Fatalf("unexpected attribute type: %T", v) + } + } + return lr +} + +// mockLogs creates mock plog.Logs with the given records. +func mockLogs(records ...plog.LogRecord) plog.Logs { + logs := plog.NewLogs() + rl := logs.ResourceLogs().AppendEmpty() + sl := rl.ScopeLogs().AppendEmpty() + for _, rec := range records { + rec.CopyTo(sl.LogRecords().AppendEmpty()) + } + return logs +} + +// mockLogRecordWithNestedBody creates a log record with a nested body structure. +func mockLogRecordWithNestedBody(body map[string]any) plog.LogRecord { + lr := plog.NewLogRecord() + lr.Body().SetEmptyMap().EnsureCapacity(len(body)) + lr.Body().Map().FromRaw(body) + return lr +} + +func TestMarshalRawLogs(t *testing.T) { + tests := []struct { + name string + logRecords []plog.LogRecord + expected []string + rawLogField string + wantErr bool + }{ + { + name: "Simple log record", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "Test body", map[string]any{"key1": "value1"}), + }, + expected: []string{`{"attributes":{"key1":"value1"},"body":"Test body","resource_attributes":{}}`}, + rawLogField: "", + wantErr: false, + }, + { + name: "Nested body log record", + logRecords: []plog.LogRecord{ + mockLogRecordWithNestedBody(map[string]any{"nested": "value"}), + }, + expected: []string{`{"attributes":{},"body":{"nested":"value"},"resource_attributes":{}}`}, + rawLogField: "", + wantErr: false, + }, + { + name: "String body log record", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "test", map[string]any{}), + }, + expected: []string{`{"attributes":{},"body":"test","resource_attributes":{}}`}, + rawLogField: "", + wantErr: false, + }, + { + name: "Invalid raw log field", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "Test body", map[string]any{"key1": "value1"}), + }, + expected: nil, + rawLogField: "invalid_field", + wantErr: true, + }, + { + name: "Valid rawLogField - simple attribute", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "Test body", map[string]any{"level": "info"}), + }, + expected: []string{"info"}, + rawLogField: `attributes["level"]`, + wantErr: false, + }, + { + name: "Valid rawLogField - nested attribute", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "Test body", map[string]any{"event": map[string]any{"type": "login"}}), + }, + expected: []string{`{"type":"login"}`}, + rawLogField: `attributes["event"]`, + wantErr: false, + }, + { + name: "Invalid rawLogField - non-existent field", + logRecords: []plog.LogRecord{ + mockLogRecord(t, "Test body", map[string]any{"key1": "value1"}), + }, + expected: nil, + rawLogField: `attributes["nonexistent"]`, + wantErr: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + core, observedLogs := observer.New(zap.InfoLevel) + logger := zap.New(core) + cfg := Config{RawLogField: tt.rawLogField} + m := newMarshaler(cfg, component.TelemetrySettings{Logger: logger}) + + logs := mockLogs(tt.logRecords...) + marshalledLogs, err := m.MarshalRawLogs(context.Background(), logs) + require.NoError(t, err) + + // Check for errors in the logs + var foundError bool + for _, log := range observedLogs.All() { + if log.Level == zap.ErrorLevel { + foundError = true + break + } + } + + if tt.wantErr { + require.True(t, foundError, "Expected an error to be logged") + } else { + require.False(t, foundError, "Did not expect an error to be logged") + + // Directly compare the marshalled strings with the expected strings + if len(tt.expected) == len(marshalledLogs) { + for i, expected := range tt.expected { + require.Equal(t, expected, marshalledLogs[i], "Marshalled log does not match expected") + } + } else { + t.Errorf("Expected %d marshalled logs, got %d", len(tt.expected), len(marshalledLogs)) + } + } + }) + } +} diff --git a/exporter/chronicleforwarderexporter/metadata.yaml b/exporter/chronicleforwarderexporter/metadata.yaml new file mode 100644 index 000000000..aa93f41dc --- /dev/null +++ b/exporter/chronicleforwarderexporter/metadata.yaml @@ -0,0 +1,7 @@ +type: chronicleforwarder + +status: + class: exporter + stability: + alpha: [logs] + distributions: [observiq] diff --git a/factories/exporters.go b/factories/exporters.go index 315d26cb7..8d7b6b550 100644 --- a/factories/exporters.go +++ b/factories/exporters.go @@ -17,6 +17,7 @@ package factories import ( "github.com/observiq/bindplane-agent/exporter/azureblobexporter" "github.com/observiq/bindplane-agent/exporter/chronicleexporter" + "github.com/observiq/bindplane-agent/exporter/chronicleforwarderexporter" "github.com/observiq/bindplane-agent/exporter/googlecloudexporter" "github.com/observiq/bindplane-agent/exporter/googlemanagedprometheusexporter" "github.com/observiq/bindplane-agent/internal/version" @@ -69,6 +70,7 @@ var defaultExporters = []exporter.Factory{ azuremonitorexporter.NewFactory(), carbonexporter.NewFactory(), chronicleexporter.NewFactory(), + chronicleforwarderexporter.NewFactory(), clickhouseexporter.NewFactory(), coralogixexporter.NewFactory(), datadogexporter.NewFactory(), diff --git a/go.mod b/go.mod index 2370593b9..41bcbbecc 100644 --- a/go.mod +++ b/go.mod @@ -7,6 +7,7 @@ require ( github.com/mholt/archiver/v3 v3.5.1 github.com/observiq/bindplane-agent/exporter/azureblobexporter v1.42.0 github.com/observiq/bindplane-agent/exporter/chronicleexporter v1.42.0 + github.com/observiq/bindplane-agent/exporter/chronicleforwarderexporter v1.42.0 github.com/observiq/bindplane-agent/exporter/googlecloudexporter v1.42.0 github.com/observiq/bindplane-agent/exporter/googlemanagedprometheusexporter v1.42.0 github.com/observiq/bindplane-agent/packagestate v1.42.0 @@ -668,6 +669,8 @@ replace github.com/observiq/bindplane-agent/exporter/azureblobexporter => ./expo replace github.com/observiq/bindplane-agent/exporter/chronicleexporter => ./exporter/chronicleexporter +replace github.com/observiq/bindplane-agent/exporter/chronicleforwarderexporter => ./exporter/chronicleforwarderexporter + replace github.com/observiq/bindplane-agent/packagestate => ./packagestate replace github.com/observiq/bindplane-agent/processor/metricstatsprocessor => ./processor/metricstatsprocessor