Targets: Add Heroku HTTPS drain target (#6448)

pull/6697/head
Pablo 4 years ago committed by GitHub
parent d2c099ac1d
commit 0e28452f1f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 8
      clients/pkg/promtail/client/fake/client.go
  2. 36
      clients/pkg/promtail/scrapeconfig/scrapeconfig.go
  3. 27
      clients/pkg/promtail/targets/heroku/metrics.go
  4. 188
      clients/pkg/promtail/targets/heroku/target.go
  5. 315
      clients/pkg/promtail/targets/heroku/target_test.go
  6. 75
      clients/pkg/promtail/targets/heroku/targetmanager.go
  7. 28
      clients/pkg/promtail/targets/manager.go
  8. 3
      clients/pkg/promtail/targets/target/target.go
  9. 40
      docs/sources/clients/promtail/configuration.md
  10. 46
      docs/sources/clients/promtail/scraping.md
  11. 1
      go.mod
  12. 73
      go.sum
  13. 27
      vendor/github.com/heroku/x/LICENSE.txt
  14. 47
      vendor/github.com/heroku/x/logplex/encoding/README.md
  15. 126
      vendor/github.com/heroku/x/logplex/encoding/encoder.go
  16. 40
      vendor/github.com/heroku/x/logplex/encoding/message.go
  17. 58
      vendor/github.com/heroku/x/logplex/encoding/parser.go
  18. 264
      vendor/github.com/heroku/x/logplex/encoding/scanner.go
  19. 3
      vendor/modules.txt

@ -60,3 +60,11 @@ func (c *Client) StopNow() {
func (c *Client) Name() string {
return "fake"
}
// Clear is used to cleanup the buffered received entries, so the same client can be re-used between
// test cases.
func (c *Client) Clear() {
c.mtx.Lock()
defer c.mtx.Unlock()
c.received = []api.Entry{}
}

@ -33,17 +33,18 @@ import (
// Config describes a job to scrape.
type Config struct {
JobName string `yaml:"job_name,omitempty"`
PipelineStages stages.PipelineStages `yaml:"pipeline_stages,omitempty"`
JournalConfig *JournalTargetConfig `yaml:"journal,omitempty"`
SyslogConfig *SyslogTargetConfig `yaml:"syslog,omitempty"`
GcplogConfig *GcplogTargetConfig `yaml:"gcplog,omitempty"`
PushConfig *PushTargetConfig `yaml:"loki_push_api,omitempty"`
WindowsConfig *WindowsEventsTargetConfig `yaml:"windows_events,omitempty"`
KafkaConfig *KafkaTargetConfig `yaml:"kafka,omitempty"`
GelfConfig *GelfTargetConfig `yaml:"gelf,omitempty"`
CloudflareConfig *CloudflareConfig `yaml:"cloudflare,omitempty"`
RelabelConfigs []*relabel.Config `yaml:"relabel_configs,omitempty"`
JobName string `yaml:"job_name,omitempty"`
PipelineStages stages.PipelineStages `yaml:"pipeline_stages,omitempty"`
JournalConfig *JournalTargetConfig `yaml:"journal,omitempty"`
SyslogConfig *SyslogTargetConfig `yaml:"syslog,omitempty"`
GcplogConfig *GcplogTargetConfig `yaml:"gcplog,omitempty"`
PushConfig *PushTargetConfig `yaml:"loki_push_api,omitempty"`
WindowsConfig *WindowsEventsTargetConfig `yaml:"windows_events,omitempty"`
KafkaConfig *KafkaTargetConfig `yaml:"kafka,omitempty"`
GelfConfig *GelfTargetConfig `yaml:"gelf,omitempty"`
CloudflareConfig *CloudflareConfig `yaml:"cloudflare,omitempty"`
HerokuDrainConfig *HerokuDrainTargetConfig `yaml:"heroku_drain,omitempty"`
RelabelConfigs []*relabel.Config `yaml:"relabel_configs,omitempty"`
// List of Docker service discovery configurations.
DockerSDConfigs []*moby.DockerSDConfig `yaml:"docker_sd_configs,omitempty"`
ServiceDiscoveryConfig ServiceDiscoveryConfig `yaml:",inline"`
@ -359,6 +360,19 @@ type GcplogTargetConfig struct {
UseIncomingTimestamp bool `yaml:"use_incoming_timestamp"`
}
// HerokuDrainTargetConfig describes a scrape config to listen and consume heroku logs, in the HTTPS drain manner.
type HerokuDrainTargetConfig struct {
// Server is the weaveworks server config for listening connections
Server server.Config `yaml:"server"`
// Labels optionally holds labels to associate with each record received on the push api.
Labels model.LabelSet `yaml:"labels"`
// UseIncomingTimestamp sets the timestamp to the incoming heroku log entry timestamp. If false,
// promtail will assign the current timestamp to the log entry when it was processed.
UseIncomingTimestamp bool `yaml:"use_incoming_timestamp"`
}
// PushTargetConfig describes a scrape config that listens for Loki push messages.
type PushTargetConfig struct {
// Server is the weaveworks server config for listening connections

@ -0,0 +1,27 @@
package heroku
import "github.com/prometheus/client_golang/prometheus"
type Metrics struct {
herokuEntries *prometheus.CounterVec
herokuErrors *prometheus.CounterVec
}
func NewMetrics(reg prometheus.Registerer) *Metrics {
var m Metrics
m.herokuEntries = prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: "promtail",
Name: "heroku_drain_target_entries_total",
Help: "Number of successful entries received by the Heroku target",
}, []string{})
m.herokuErrors = prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: "promtail",
Name: "heroku_drain_target_parsing_errors_total",
Help: "Number of parsing errors while receiving Heroku messages",
}, []string{})
reg.MustRegister(m.herokuEntries, m.herokuErrors)
return &m
}

@ -0,0 +1,188 @@
package heroku
import (
"flag"
"fmt"
"net/http"
"strings"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
herokuEncoding "github.com/heroku/x/logplex/encoding"
"github.com/imdario/mergo"
"github.com/pkg/errors"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/model/relabel"
"github.com/weaveworks/common/logging"
"github.com/weaveworks/common/server"
"github.com/grafana/loki/clients/pkg/promtail/api"
"github.com/grafana/loki/clients/pkg/promtail/scrapeconfig"
"github.com/grafana/loki/clients/pkg/promtail/targets/target"
"github.com/grafana/loki/pkg/logproto"
util_log "github.com/grafana/loki/pkg/util/log"
)
type Target struct {
logger log.Logger
handler api.EntryHandler
config *scrapeconfig.HerokuDrainTargetConfig
jobName string
server *server.Server
metrics *Metrics
relabelConfigs []*relabel.Config
}
// NewTarget creates a brand new Heroku Drain target, capable of receiving logs from a Heroku application through an HTTP drain.
func NewTarget(metrics *Metrics, logger log.Logger, handler api.EntryHandler, jobName string, config *scrapeconfig.HerokuDrainTargetConfig, relabel []*relabel.Config) (*Target, error) {
wrappedLogger := log.With(logger, "component", "heroku_drain")
ht := &Target{
metrics: metrics,
logger: wrappedLogger,
handler: handler,
jobName: jobName,
config: config,
relabelConfigs: relabel,
}
// Bit of a chicken and egg problem trying to register the defaults and apply overrides from the loaded config.
// First create an empty config and set defaults.
defaults := server.Config{}
defaults.RegisterFlags(flag.NewFlagSet("empty", flag.ContinueOnError))
// Then apply any config values loaded as overrides to the defaults.
if err := mergo.Merge(&defaults, config.Server, mergo.WithOverride); err != nil {
return nil, errors.Wrap(err, "failed to parse configs and override defaults when configuring heroku drain target")
}
// The merge won't overwrite with a zero value but in the case of ports 0 value
// indicates the desire for a random port so reset these to zero if the incoming config val is 0
if config.Server.HTTPListenPort == 0 {
defaults.HTTPListenPort = 0
}
if config.Server.GRPCListenPort == 0 {
defaults.GRPCListenPort = 0
}
// Set the config to the new combined config.
config.Server = defaults
err := ht.run()
if err != nil {
return nil, err
}
return ht, nil
}
func (h *Target) run() error {
level.Info(h.logger).Log("msg", "starting heroku drain target", "job", h.jobName)
// To prevent metric collisions because all metrics are going to be registered in the global Prometheus registry.
tentativeServerMetricNamespace := "promtail_heroku_drain_target_" + h.jobName
if !model.IsValidMetricName(model.LabelValue(tentativeServerMetricNamespace)) {
return fmt.Errorf("invalid prometheus-compatible job name: %s", h.jobName)
}
h.config.Server.MetricsNamespace = tentativeServerMetricNamespace
// We don't want the /debug and /metrics endpoints running, since this is not the main promtail HTTP server.
// We want this target to expose the least surface area possible, hence disabling WeaveWorks HTTP server metrics
// and debugging functionality.
h.config.Server.RegisterInstrumentation = false
// Wrapping util logger with component-specific key vals, and the expected GoKit logging interface
h.config.Server.Log = logging.GoKit(log.With(util_log.Logger, "component", "heroku_drain"))
srv, err := server.New(h.config.Server)
if err != nil {
return err
}
h.server = srv
h.server.HTTP.Path("/heroku/api/v1/drain").Methods("POST").Handler(http.HandlerFunc(h.drain))
go func() {
err := srv.Run()
if err != nil {
level.Error(h.logger).Log("msg", "heroku drain target shutdown with error", "err", err)
}
}()
return nil
}
func (h *Target) drain(w http.ResponseWriter, r *http.Request) {
entries := h.handler.Chan()
defer r.Body.Close()
herokuScanner := herokuEncoding.NewDrainScanner(r.Body)
for herokuScanner.Scan() {
ts := time.Now()
message := herokuScanner.Message()
lb := labels.NewBuilder(nil)
lb.Set("__heroku_drain_host", message.Hostname)
lb.Set("__heroku_drain_app", message.Application)
lb.Set("__heroku_drain_proc", message.Process)
lb.Set("__heroku_drain_log_id", message.ID)
if h.config.UseIncomingTimestamp {
ts = message.Timestamp
}
processed := relabel.Process(lb.Labels(), h.relabelConfigs...)
// Start with the set of labels fixed in the configuration
filtered := h.Labels().Clone()
for _, lbl := range processed {
if strings.HasPrefix(lbl.Name, "__") {
continue
}
filtered[model.LabelName(lbl.Name)] = model.LabelValue(lbl.Value)
}
entries <- api.Entry{
Labels: filtered,
Entry: logproto.Entry{
Timestamp: ts,
Line: message.Message,
},
}
h.metrics.herokuEntries.WithLabelValues().Inc()
}
err := herokuScanner.Err()
if err != nil {
h.metrics.herokuErrors.WithLabelValues().Inc()
level.Warn(h.logger).Log("msg", "failed to read incoming heroku request", "err", err.Error())
http.Error(w, err.Error(), http.StatusBadRequest)
return
}
w.WriteHeader(http.StatusNoContent)
}
func (h *Target) Type() target.TargetType {
return target.HerokuDrainTargetType
}
func (h *Target) DiscoveredLabels() model.LabelSet {
return nil
}
func (h *Target) Labels() model.LabelSet {
return h.config.Labels
}
func (h *Target) Ready() bool {
return true
}
func (h *Target) Details() interface{} {
return map[string]string{}
}
func (h *Target) Stop() error {
level.Info(h.logger).Log("msg", "stopping heroku drain target", "job", h.jobName)
h.server.Shutdown()
h.handler.Stop()
return nil
}

@ -0,0 +1,315 @@
package heroku
import (
"flag"
"fmt"
"net"
"net/http"
"os"
"strings"
"testing"
"time"
"github.com/go-kit/log"
"github.com/google/uuid"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/prometheus/prometheus/model/relabel"
"github.com/stretchr/testify/require"
"github.com/weaveworks/common/server"
"github.com/grafana/loki/clients/pkg/promtail/client/fake"
"github.com/grafana/loki/clients/pkg/promtail/scrapeconfig"
)
const localhost = "127.0.0.1"
const testPayload = `270 <158>1 2022-06-13T14:52:23.622778+00:00 host heroku router - at=info method=GET path="/" host=cryptic-cliffs-27764.herokuapp.com request_id=59da6323-2bc4-4143-8677-cc66ccfb115f fwd="181.167.87.140" dyno=web.1 connect=0ms service=3ms status=200 bytes=6979 protocol=https
`
const testLogLine1 = `140 <190>1 2022-06-13T14:52:23.621815+00:00 host app web.1 - [GIN] 2022/06/13 - 14:52:23 | 200 | 1.428101ms | 181.167.87.140 | GET "/"
`
const testLogLine1Timestamp = "2022-06-13T14:52:23.621815+00:00"
const testLogLine2 = `156 <190>1 2022-06-13T14:52:23.827271+00:00 host app web.1 - [GIN] 2022/06/13 - 14:52:23 | 200 | 163.92µs | 181.167.87.140 | GET "/static/main.css"
`
func makeDrainRequest(host string, bodies ...string) (*http.Request, error) {
req, err := http.NewRequest(http.MethodPost, fmt.Sprintf("%s/heroku/api/v1/drain", host), strings.NewReader(strings.Join(bodies, "")))
if err != nil {
return nil, err
}
drainToken := uuid.New().String()
frameID := uuid.New().String()
req.Header.Set("Content-Type", "application/heroku_drain-1")
req.Header.Set("Logplex-Drain-Token", fmt.Sprintf("d.%s", drainToken))
req.Header.Set("Logplex-Frame-Id", frameID)
req.Header.Set("Logplex-Msg-Count", fmt.Sprintf("%d", len(bodies)))
return req, nil
}
func TestHerokuDrainTarget(t *testing.T) {
w := log.NewSyncWriter(os.Stderr)
logger := log.NewLogfmtLogger(w)
type expectedEntry struct {
labels model.LabelSet
line string
}
type args struct {
RequestBodies []string
RelabelConfigs []*relabel.Config
Labels model.LabelSet
}
cases := map[string]struct {
args args
expectedEntries []expectedEntry
}{
"heroku request with a single log line, internal labels dropped, and fixed are propagated": {
args: args{
RequestBodies: []string{testPayload},
Labels: model.LabelSet{
"job": "some_job_name",
},
},
expectedEntries: []expectedEntry{
{
labels: model.LabelSet{
"job": "some_job_name",
},
line: `at=info method=GET path="/" host=cryptic-cliffs-27764.herokuapp.com request_id=59da6323-2bc4-4143-8677-cc66ccfb115f fwd="181.167.87.140" dyno=web.1 connect=0ms service=3ms status=200 bytes=6979 protocol=https
`,
},
},
},
"heroku request with a two log lines, internal labels dropped, and fixed are propagated": {
args: args{
RequestBodies: []string{testLogLine1, testLogLine2},
Labels: model.LabelSet{
"job": "multiple_line_job",
},
},
expectedEntries: []expectedEntry{
{
labels: model.LabelSet{
"job": "multiple_line_job",
},
line: `[GIN] 2022/06/13 - 14:52:23 | 200 | 1.428101ms | 181.167.87.140 | GET "/"
`,
},
{
labels: model.LabelSet{
"job": "multiple_line_job",
},
line: `[GIN] 2022/06/13 - 14:52:23 | 200 | 163.92µs | 181.167.87.140 | GET "/static/main.css"
`,
},
},
},
"heroku request with a single log line, with internal labels relabeled, and fixed labels": {
args: args{
RequestBodies: []string{testLogLine1},
Labels: model.LabelSet{
"job": "relabeling_job",
},
RelabelConfigs: []*relabel.Config{
{
SourceLabels: model.LabelNames{"__heroku_drain_host"},
TargetLabel: "host",
Replacement: "$1",
Action: relabel.Replace,
Regex: relabel.MustNewRegexp("(.*)"),
},
{
SourceLabels: model.LabelNames{"__heroku_drain_app"},
TargetLabel: "app",
Replacement: "$1",
Action: relabel.Replace,
Regex: relabel.MustNewRegexp("(.*)"),
},
{
SourceLabels: model.LabelNames{"__heroku_drain_proc"},
TargetLabel: "procID",
Replacement: "$1",
Action: relabel.Replace,
Regex: relabel.MustNewRegexp("(.*)"),
},
},
},
expectedEntries: []expectedEntry{
{
line: `[GIN] 2022/06/13 - 14:52:23 | 200 | 1.428101ms | 181.167.87.140 | GET "/"
`,
labels: model.LabelSet{
"host": "host",
"app": "app",
"procID": "web.1",
},
},
},
},
}
for name, tc := range cases {
t.Run(name, func(t *testing.T) {
// Create fake promtail client
eh := fake.New(func() {})
defer eh.Stop()
serverConfig, port, err := getServerConfigWithAvailablePort()
require.NoError(t, err, "error generating server config or finding open port")
config := &scrapeconfig.HerokuDrainTargetConfig{
Server: serverConfig,
Labels: tc.args.Labels,
UseIncomingTimestamp: false,
}
prometheus.DefaultRegisterer = prometheus.NewRegistry()
metrics := NewMetrics(prometheus.DefaultRegisterer)
pt, err := NewTarget(metrics, logger, eh, "test_job", config, tc.args.RelabelConfigs)
require.NoError(t, err)
defer func() {
_ = pt.Stop()
}()
// Clear received lines after test case is ran
defer eh.Clear()
// Send some logs
ts := time.Now()
req, err := makeDrainRequest(fmt.Sprintf("http://%s:%d", localhost, port), tc.args.RequestBodies...)
require.NoError(t, err, "expected test drain request to be successfully created")
res, err := http.DefaultClient.Do(req)
require.NoError(t, err)
require.Equal(t, http.StatusNoContent, res.StatusCode, "expected no-content status code")
// Wait for them to appear in the test handler
countdown := 1000
for len(eh.Received()) != 1 && countdown > 0 {
time.Sleep(1 * time.Millisecond)
countdown--
}
// Make sure we didn't timeout
require.Equal(t, len(tc.args.RequestBodies), len(eh.Received()))
require.Equal(t, len(eh.Received()), len(tc.expectedEntries), "expected to receive equal amount of expected label sets")
for i, expectedEntry := range tc.expectedEntries {
// TODO: Add assertion over propagated timestamp
actualEntry := eh.Received()[i]
require.Equal(t, expectedEntry.line, actualEntry.Line, "expected line to be equal for %d-th entry", i)
expectedLS := expectedEntry.labels
actualLS := actualEntry.Labels
for label, value := range expectedLS {
require.Equal(t, expectedLS[label], actualLS[label], "expected label %s to be equal to %s in %d-th entry", label, value, i)
}
// Timestamp is always set in the handler, we expect received timestamps to be slightly higher than the timestamp when we started sending logs.
require.GreaterOrEqual(t, actualEntry.Timestamp.Unix(), ts.Unix(), "expected %d-th entry to have a received timestamp greater than publish time", i)
}
})
}
}
func TestHerokuDrainTarget_UseIncomingTimestamp(t *testing.T) {
w := log.NewSyncWriter(os.Stderr)
logger := log.NewLogfmtLogger(w)
// Create fake promtail client
eh := fake.New(func() {})
defer eh.Stop()
serverConfig, port, err := getServerConfigWithAvailablePort()
require.NoError(t, err, "error generating server config or finding open port")
config := &scrapeconfig.HerokuDrainTargetConfig{
Server: serverConfig,
Labels: nil,
UseIncomingTimestamp: true,
}
prometheus.DefaultRegisterer = prometheus.NewRegistry()
metrics := NewMetrics(prometheus.DefaultRegisterer)
pt, err := NewTarget(metrics, logger, eh, "test_job", config, nil)
require.NoError(t, err)
defer func() {
_ = pt.Stop()
}()
// Clear received lines after test case is ran
defer eh.Clear()
req, err := makeDrainRequest(fmt.Sprintf("http://%s:%d", localhost, port), testLogLine1)
require.NoError(t, err, "expected test drain request to be successfully created")
res, err := http.DefaultClient.Do(req)
require.NoError(t, err)
require.Equal(t, http.StatusNoContent, res.StatusCode, "expected no-content status code")
// Wait for them to appear in the test handler
countdown := 1000
for len(eh.Received()) != 1 && countdown > 0 {
time.Sleep(1 * time.Millisecond)
countdown--
}
// Make sure we didn't timeout
require.Equal(t, 1, len(eh.Received()))
expectedTs, err := time.Parse(time.RFC3339Nano, testLogLine1Timestamp)
require.NoError(t, err, "expected expected timestamp to be parse correctly")
require.Equal(t, expectedTs, eh.Received()[0].Timestamp, "expected entry timestamp to be overridden by received one")
}
func TestHerokuDrainTarget_ErrorOnNotPrometheusCompatibleJobName(t *testing.T) {
w := log.NewSyncWriter(os.Stderr)
logger := log.NewLogfmtLogger(w)
// Create fake promtail client
eh := fake.New(func() {})
defer eh.Stop()
serverConfig, _, err := getServerConfigWithAvailablePort()
require.NoError(t, err, "error generating server config or finding open port")
config := &scrapeconfig.HerokuDrainTargetConfig{
Server: serverConfig,
Labels: nil,
UseIncomingTimestamp: true,
}
prometheus.DefaultRegisterer = prometheus.NewRegistry()
metrics := NewMetrics(prometheus.DefaultRegisterer)
pt, err := NewTarget(metrics, logger, eh, "test-job", config, nil)
require.Error(t, err, "expected an error from creating a heroku target with an invalid job name")
// Cleanup target in the case test failed and target started correctly
if err == nil {
_ = pt.Stop()
}
}
func getServerConfigWithAvailablePort() (cfg server.Config, port int, err error) {
// Get a randomly available port by open and closing a TCP socket
addr, err := net.ResolveTCPAddr("tcp", localhost+":0")
if err != nil {
return
}
l, err := net.ListenTCP("tcp", addr)
if err != nil {
return
}
port = l.Addr().(*net.TCPAddr).Port
err = l.Close()
if err != nil {
return
}
// Adjust some of the defaults
cfg.RegisterFlags(flag.NewFlagSet("empty", flag.ContinueOnError))
cfg.HTTPListenAddress = localhost
cfg.HTTPListenPort = port
cfg.GRPCListenAddress = localhost
cfg.GRPCListenPort = 0 // Not testing GRPC, a random port will be assigned
return
}

@ -0,0 +1,75 @@
package heroku
import (
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/prometheus/client_golang/prometheus"
"github.com/grafana/loki/clients/pkg/logentry/stages"
"github.com/grafana/loki/clients/pkg/promtail/api"
"github.com/grafana/loki/clients/pkg/promtail/scrapeconfig"
"github.com/grafana/loki/clients/pkg/promtail/targets/target"
)
type TargetManager struct {
logger log.Logger
targets map[string]*Target
}
func NewHerokuDrainTargetManager(
metrics *Metrics,
reg prometheus.Registerer,
logger log.Logger,
client api.EntryHandler,
scrapeConfigs []scrapeconfig.Config) (*TargetManager, error) {
tm := &TargetManager{
logger: logger,
targets: make(map[string]*Target),
}
for _, cfg := range scrapeConfigs {
pipeline, err := stages.NewPipeline(log.With(logger, "component", "heroku_drain_pipeline_"+cfg.JobName), cfg.PipelineStages, &cfg.JobName, reg)
if err != nil {
return nil, err
}
t, err := NewTarget(metrics, logger, pipeline.Wrap(client), cfg.JobName, cfg.HerokuDrainConfig, cfg.RelabelConfigs)
if err != nil {
return nil, err
}
tm.targets[cfg.JobName] = t
}
return tm, nil
}
func (hm *TargetManager) Ready() bool {
for _, t := range hm.targets {
if t.Ready() {
return true
}
}
return false
}
func (hm *TargetManager) Stop() {
for name, t := range hm.targets {
if err := t.Stop(); err != nil {
level.Error(t.logger).Log("event", "failed to stop heroku drain target", "name", name, "cause", err)
}
}
}
func (hm *TargetManager) ActiveTargets() map[string][]target.Target {
return hm.AllTargets()
}
func (hm *TargetManager) AllTargets() map[string][]target.Target {
res := make(map[string][]target.Target, len(hm.targets))
for k, v := range hm.targets {
res[k] = []target.Target{v}
}
return res
}

@ -16,6 +16,7 @@ import (
"github.com/grafana/loki/clients/pkg/promtail/targets/file"
"github.com/grafana/loki/clients/pkg/promtail/targets/gcplog"
"github.com/grafana/loki/clients/pkg/promtail/targets/gelf"
"github.com/grafana/loki/clients/pkg/promtail/targets/heroku"
"github.com/grafana/loki/clients/pkg/promtail/targets/journal"
"github.com/grafana/loki/clients/pkg/promtail/targets/kafka"
"github.com/grafana/loki/clients/pkg/promtail/targets/lokipush"
@ -37,6 +38,7 @@ const (
CloudflareConfigs = "cloudflareConfigs"
DockerConfigs = "dockerConfigs"
DockerSDConfigs = "dockerSDConfigs"
HerokuDrainConfigs = "herokuDrainConfigs"
)
type targetManager interface {
@ -96,6 +98,8 @@ func NewTargetManagers(
targetScrapeConfigs[CloudflareConfigs] = append(targetScrapeConfigs[CloudflareConfigs], cfg)
case cfg.DockerSDConfigs != nil:
targetScrapeConfigs[DockerSDConfigs] = append(targetScrapeConfigs[DockerSDConfigs], cfg)
case cfg.HerokuDrainConfig != nil:
targetScrapeConfigs[HerokuDrainConfigs] = append(targetScrapeConfigs[HerokuDrainConfigs], cfg)
default:
return nil, fmt.Errorf("no valid target scrape config defined for %q", cfg.JobName)
}
@ -116,13 +120,14 @@ func NewTargetManagers(
}
var (
fileMetrics *file.Metrics
syslogMetrics *syslog.Metrics
gcplogMetrics *gcplog.Metrics
gelfMetrics *gelf.Metrics
cloudflareMetrics *cloudflare.Metrics
dockerMetrics *docker.Metrics
journalMetrics *journal.Metrics
fileMetrics *file.Metrics
syslogMetrics *syslog.Metrics
gcplogMetrics *gcplog.Metrics
gelfMetrics *gelf.Metrics
cloudflareMetrics *cloudflare.Metrics
dockerMetrics *docker.Metrics
journalMetrics *journal.Metrics
herokuDrainMetrics *heroku.Metrics
)
if len(targetScrapeConfigs[FileScrapeConfigs]) > 0 {
fileMetrics = file.NewMetrics(reg)
@ -145,6 +150,9 @@ func NewTargetManagers(
if len(targetScrapeConfigs[JournalScrapeConfigs]) > 0 {
journalMetrics = journal.NewMetrics(reg)
}
if len(targetScrapeConfigs[HerokuDrainConfigs]) > 0 {
herokuDrainMetrics = heroku.NewMetrics(reg)
}
for target, scrapeConfigs := range targetScrapeConfigs {
switch target {
@ -214,6 +222,12 @@ func NewTargetManagers(
return nil, errors.Wrap(err, "failed to make Loki Push API target manager")
}
targetManagers = append(targetManagers, pushTargetManager)
case HerokuDrainConfigs:
herokuDrainTargetManager, err := heroku.NewHerokuDrainTargetManager(herokuDrainMetrics, reg, logger, client, scrapeConfigs)
if err != nil {
return nil, errors.Wrap(err, "failed to make Heroku drain target manager")
}
targetManagers = append(targetManagers, herokuDrainTargetManager)
case WindowsEventsConfigs:
windowsTargetManager, err := windows.NewTargetManager(reg, logger, client, scrapeConfigs)
if err != nil {

@ -41,6 +41,9 @@ const (
// DockerTargetType is a Docker target
DockerTargetType = TargetType("Docker")
// HerokuDrainTargetType is a Heroku Logs target
HerokuDrainTargetType = TargetType("HerokuDrain")
)
// Target is a promtail scrape target

@ -339,6 +339,9 @@ job_name: <string>
# Configuration describing how to pull logs from Cloudflare.
[cloudflare: <cloudflare>]
# Configuration describing how to pull logs from a Heroku LogPlex drain.
[heroku_drain: <heroku_drain>]
# Describes how to relabel targets to determine if they should
# be processed.
relabel_configs:
@ -1194,6 +1197,43 @@ All Cloudflare logs are in JSON. Here is an example:
You can leverage [pipeline stages](pipeline_stages) if, for example, you want to parse the JSON log line and extract more labels or change the log line format.
### heroku_drain
The `heroku_drain` block configures Promtail to expose a [Heroku HTTPS Drain](https://devcenter.heroku.com/articles/log-drains#https-drains).
Each job configured with a Heroku Drain will expose a Drain and will require a separate port.
The `server` configuration is the same as [server](#server), since Promtail exposes an HTTP server for each new drain.
Promtail exposes an endpoint at `/heroku/api/v1/drain`, which expects requests from Heroku's log delivery.
```yaml
# The Heroku drain server configuration options
[server: <server_config>]
# Label map to add to every log message.
labels:
[ <labelname>: <labelvalue> ... ]
# Whether Promtail should pass on the timestamp from the incoming Heroku drain message.
# When false, or if no timestamp is present in the syslog message, Promtail will assign the current
# timestamp to the log when it was processed.
[use_incoming_timestamp: <boolean> | default = false]
```
#### Available Labels
Heroku Log drains send logs in [Syslog-formatted messages](https://datatracker.ietf.org/doc/html/rfc5424#section-6) (with
some [minor tweaks](https://devcenter.heroku.com/articles/log-drains#https-drain-caveats); they are not RFC-compatible).
The Heroku Drain target exposes for each log entry the received syslog fields with the following labels:
- `__heroku_drain_host`: The [HOSTNAME](https://tools.ietf.org/html/rfc5424#section-6.2.4) field parsed from the message.
- `__heroku_drain_app`: The [APP-NAME](https://tools.ietf.org/html/rfc5424#section-6.2.5) field parsed from the message.
- `__heroku_drain_proc`: The [PROCID](https://tools.ietf.org/html/rfc5424#section-6.2.6) field parsed from the message.
- `__heroku_drain_log_id`: The [MSGID](https://tools.ietf.org/html/rfc5424#section-6.2.7) field parsed from the message.
### relabel_configs
Relabeling is a powerful tool to dynamically rewrite the label set of a target

@ -379,6 +379,52 @@ scrape_configs:
Only `api_token` and `zone_id` are required.
Refer to the [Cloudfare](../../configuration/#cloudflare) configuration section for details.
## Heroku Drain
Promtail supports receiving logs from a Heroku application by using a [Heroku HTTPS Drain](https://devcenter.heroku.com/articles/log-drains#https-drains).
Configuration is specified in a`heroku_drain` block within the Promtail `scrape_config` configuration.
```yaml
- job_name: heroku_drain
heroku_drain:
server:
http_listen_address: 0.0.0.0
http_listen_port: 8080
labels:
job: heroku_drain_docs
use_incoming_timestamp: true
relabel_configs:
- source_labels: ['__heroku_drain_host']
target_label: 'host'
- source_labels: ['__heroku_drain_app']
target_label: 'app'
- source_labels: ['__heroku_drain_proc']
target_label: 'proc'
- source_labels: ['__heroku_drain_log_id']
target_label: 'log_id'
```
Within the `scrape_configs` configuration for a Heroku Drain target, the `job_name` must be a Prometheus-compatible [metric name](https://prometheus.io/docs/concepts/data_model/#metric-names-and-labels).
The [server](../configuration.md#server) section configures the HTTP server created for receiving logs.
`labels` defines a static set of label values added to each received log entry. `use_incoming_timestamp` can be used to pass
the timestamp received from Heroku.
Before using a `heroku_drain` target, Heroku should be configured with the URL where the Promtail instance will be listening.
Follow the steps in [Heroku HTTPS Drain docs](https://devcenter.heroku.com/articles/log-drains#https-drains) for using the Heroku CLI
with a command like the following:
```
heroku drains:add [http|https]://HOSTNAME:8080/heroku/api/v1/drain -a HEROKU_APP_NAME
```
It also supports `relabeling` and `pipeline` stages just like other targets.
When Promtail receives Heroku Drain logs, various internal labels are made available for [relabeling](#relabeling):
- `__heroku_drain_host`
- `__heroku_drain_app`
- `__heroku_drain_proc`
- `__heroku_drain_log_id`
In the example above, the `project_id` label from a GCP resource was transformed into a label called `project` through `relabel_configs`.
## Relabeling
Each `scrape_configs` entry can contain a `relabel_configs` stanza.

@ -114,6 +114,7 @@ require (
require (
github.com/grafana/groupcache_exporter v0.0.0-20220629095919-59a8c6428a43
github.com/heroku/x v0.0.50
github.com/mailgun/groupcache/v2 v2.3.2
github.com/prometheus/alertmanager v0.24.0
github.com/prometheus/common/sigv4 v0.1.0

@ -65,6 +65,7 @@ cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9
cloud.google.com/go/trace v0.1.0/go.mod h1:wxEwsoeRVPbeSkt7ZC9nWCgmoKQRAoySN7XHW2AmI7g=
code.cloudfoundry.org/clock v1.0.0/go.mod h1:QD9Lzhd/ux6eNQVUDVRJX/RKTigpewimNYBi7ivZKY8=
collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE=
contrib.go.opencensus.io/exporter/ocagent v0.6.0/go.mod h1:zmKjrJcdo0aYcVS7bmEeSEBLPA9YJp5bjrofdU3pIXs=
dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU=
github.com/Azure/azure-amqp-common-go/v3 v3.0.0/go.mod h1:SY08giD/XbhTz07tJdpw1SoxQXHPN30+DI3Z04SYqyg=
github.com/Azure/azure-event-hubs-go/v3 v3.2.0/go.mod h1:BPIIJNH/l/fVHYq3Rm6eg4clbrULrQ3q7+icmqHyyLc=
@ -177,6 +178,7 @@ github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMx
github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae h1:ePgznFqEG1v3AjMklnK8H7BSc++FDSo7xfK9K7Af+0Y=
github.com/Shopify/toxiproxy/v2 v2.1.6-0.20210914104332-15ea381dcdae/go.mod h1:/cvHQkZ1fst0EmZnA5dFtiQdWCNCFYzb+uE2vqVgvx0=
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg=
github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8=
github.com/VividCortex/gohistogram v1.0.0/go.mod h1:Pf5mBqqDxYaXu3hDrrU+w6nw50o/4+TcAqDqk/vUH7g=
github.com/Workiva/go-datastructures v1.0.53 h1:J6Y/52yX10Xc5JjXmGtWoSSxs3mZnGSaq37xZZh7Yig=
github.com/Workiva/go-datastructures v1.0.53/go.mod h1:1yZL+zfsztete+ePzZz/Zb1/t5BnDuE2Ya2MMGhzP6A=
@ -214,6 +216,7 @@ github.com/armon/go-metrics v0.3.0/go.mod h1:zXjbSimjXTd7vOpY8B0/2LpvNvDoXBuplAD
github.com/armon/go-metrics v0.3.3/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
github.com/armon/go-metrics v0.3.9 h1:O2sNqxBdvq8Eq5xmzljcYzAORli6RWCvEym4cJf9m18=
github.com/armon/go-metrics v0.3.9/go.mod h1:4O98XIr/9W0sxpJ8UaYkvjk10Iff7SnFrb4QAOwNTFc=
github.com/armon/go-proxyproto v0.0.0-20190211145416-68259f75880e/go.mod h1:QmP9hvJ91BbJmGVGSbutW19IC0Q9phDCLGaomwTJbgU=
github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5/go.mod h1:wHh0iHkYZB8zMSxRWpUBQtwG5a7fFgvEO+odwuTv2gs=
@ -227,6 +230,8 @@ github.com/asaskevich/govalidator v0.0.0-20200907205600-7a23bdc65eef/go.mod h1:W
github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d h1:Byv0BzEl3/e6D5CLfI0j/7hiIEtvGVFPCZ7Ei2oq8iQ=
github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw=
github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU=
github.com/aws/aws-lambda-go v1.27.0/go.mod h1:jJmlefzPfGnckuHdXX7/80O3BvUUi12XOkbv4w9SGLU=
github.com/aws/aws-sdk-go v1.13.10/go.mod h1:ZRmQr0FajVIyZ4ZzBYKG5P3ZqPz9IHG41ZoMu1ADI3k=
github.com/aws/aws-sdk-go v1.15.24/go.mod h1:mFuSZ37Z9YOHbQEwBWztmVzqXrEkub65tZoCYDt7FT0=
github.com/aws/aws-sdk-go v1.17.7/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
@ -243,6 +248,7 @@ github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZw
github.com/aws/aws-sdk-go-v2 v1.9.1/go.mod h1:cK/D0BBs0b/oWPIcX/Z/obahJK1TT7IPVjy53i/mX/4=
github.com/aws/aws-sdk-go-v2/service/cloudwatch v1.8.1/go.mod h1:CM+19rL1+4dFWnOQKwDc7H1KwXTz+h61oUSHyhV0b3o=
github.com/aws/smithy-go v1.8.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E=
github.com/axiomhq/hyperloglog v0.0.0-20180317131949-fe9507de0228/go.mod h1:IOXAcuKIFq/mDyuQ4wyJuJ79XLMsmLM+5RdQ+vWrL7o=
github.com/baidubce/bce-sdk-go v0.9.81 h1:n8KfThLG9fvGv3A+RtTt/jKhg/FPPRpo+iNnS2r+iPI=
github.com/baidubce/bce-sdk-go v0.9.81/go.mod h1:zbYJMQwE4IZuyrJiFO8tO8NbtYiKTFTbwh4eIsqjVdg=
github.com/beevik/ntp v0.2.0/go.mod h1:hIHWr+l3+/clUnF44zdK+CWW7fO8dR5cIylAQ76NRpg=
@ -310,6 +316,7 @@ github.com/cockroachdb/apd v1.1.0/go.mod h1:8Sl8LxpKi29FqWXR16WEFZRNSz3SoPzUzeMe
github.com/cockroachdb/cockroach-go v0.0.0-20181001143604-e0a95dfd547c/go.mod h1:XGLbWH/ujMcbPbhZq52Nv6UrCghb1yGn//133kEsvDk=
github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8=
github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI=
github.com/codegangsta/negroni v1.0.0/go.mod h1:v0y3T5G7Y1UlFfyxFn/QLRU4a2EuNau2iZY63YTKWo0=
github.com/containerd/cgroups v1.0.1/go.mod h1:0SJrPIenamHDcZhEcJMNBB85rHcUsw4f25ZfBiPYRkU=
github.com/containerd/containerd v1.2.7/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
github.com/containerd/containerd v1.4.1/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
@ -369,6 +376,7 @@ github.com/denverdino/aliyungo v0.0.0-20170926055100-d3308649c661/go.mod h1:dV8l
github.com/devigned/tab v0.1.1/go.mod h1:XG9mPq0dFghrYvoBF3xdRrJzSTX1b7IQrvaL9mzjeJY=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
github.com/dgrijalva/jwt-go/v4 v4.0.0-preview1/go.mod h1:+hnT3ywWDTAFrW5aE+u2Sa/wT555ZqwoCS+pk3p6ry4=
github.com/dgryski/go-metro v0.0.0-20180109044635-280f6062b5bc/go.mod h1:c9O8+fpSOX1DM8cPNSkX/qsBWdkD4yd2dpciOWQjpBw=
github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78=
github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc=
github.com/dgryski/go-sip13 v0.0.0-20200911182023-62edffca9245/go.mod h1:vAd38F8PWV+bWy6jNmig1y/TA+kYO4g3RSRF0IAv0no=
@ -489,10 +497,12 @@ github.com/gin-gonic/gin v1.5.0/go.mod h1:Nd6IXA8m5kNZdNEHMBd93KT+mdY3+bewLgRvmC
github.com/glinton/ping v0.1.4-0.20200311211934-5ac87da8cd96/go.mod h1:uY+1eqFUyotrQxF1wYFNtMeHp/swbYRsoGzfcPZ8x3o=
github.com/globalsign/mgo v0.0.0-20180905125535-1ca0a4f7cbcb/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q=
github.com/globalsign/mgo v0.0.0-20181015135952-eeefdecb41b8/go.mod h1:xkRDCp4j0OGD1HRkm4kmhM+pmpv3AKq5SU7GMg4oO/Q=
github.com/go-chi/chi v4.1.2+incompatible/go.mod h1:eB3wogJHnLi3x/kFX2A+IbTBlXxmMeXJVKy9tTv1XzQ=
github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
github.com/go-ini/ini v1.25.4/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-ini/ini v1.33.0/go.mod h1:ByCAeIL28uOIIG0E3PJtZPDL8WnHpFKFOtgjp+3Ies8=
github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
github.com/go-kit/kit v0.9.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as=
github.com/go-kit/kit v0.10.0/go.mod h1:xUsJbQ/Fp4kEt7AFgCuvyX4a71u8h9jB8tj/ORgOZ7o=
@ -517,6 +527,8 @@ github.com/go-logr/logr v1.2.3/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbV
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/go-ole/go-ole v1.2.1/go.mod h1:7FAglXiTm7HKlQRDeOQ6ZNUHidzCWXuZWq/1dTyBNF8=
github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0=
github.com/go-ole/go-ole v1.2.6-0.20210915003542-8b1f7f90f6b1/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0=
github.com/go-openapi/analysis v0.0.0-20180825180245-b006789cd277/go.mod h1:k70tL6pCuVxPJOHXQ+wIac1FUrvNkHolPie/cLEU6hI=
github.com/go-openapi/analysis v0.17.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik=
github.com/go-openapi/analysis v0.18.0/go.mod h1:IowGgpVeD0vNm45So8nr+IcQ3pxVtpRoBWb8PVZO0ik=
@ -742,6 +754,7 @@ github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW
github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM=
github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
github.com/gomodule/redigo v1.8.1/go.mod h1:P9dn9mFrCBvWhGE1wpxx6fgq7BAeLBk+UUUzlpkBYO0=
github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ=
github.com/google/btree v1.0.1 h1:gK4Kx5IaGY9CD5sPJ36FHiBJ6ZXl0kilRiiCj+jdYp4=
@ -773,6 +786,7 @@ github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/
github.com/google/gofuzz v1.1.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/gofuzz v1.2.0 h1:xRy4A+RhZaiKjJ1bPfwQ8sedCA+YS2YcCHW6ec7JMi0=
github.com/google/gofuzz v1.2.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg=
github.com/google/gops v0.3.22/go.mod h1:7diIdLsqpCihPSX3fQagksT/Ku/y4RL9LHTlKyEUDl8=
github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no=
github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs=
github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0=
@ -857,6 +871,7 @@ github.com/grafana/tail v0.0.0-20220426200921-98e8eb28ea4c/go.mod h1:GIMXMPB/lRA
github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA=
github.com/grpc-ecosystem/go-grpc-middleware v1.0.1-0.20190118093823-f849b5445de4/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs=
github.com/grpc-ecosystem/go-grpc-middleware v1.1.0/go.mod h1:f5nM7jw/oeRSadq3xCzHAvxcr8HZnzsqU6ILg/0NiiE=
github.com/grpc-ecosystem/go-grpc-middleware v1.2.0/go.mod h1:mJzapYve32yjrKlk9GbyCZHuPgZsrbyIbyKhSzOpg6s=
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 h1:+9834+KizmvFV7pXQGSXQTsaWhq2GjuNUt0aUU0YBYw=
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0/go.mod h1:z0ButlSOZa5vEBq9m2m2hlwIgKw+rp3sdCBRoJY+30Y=
github.com/grpc-ecosystem/go-grpc-middleware/providers/kit/v2 v2.0.0-20201002093600-73cf2ae9d891/go.mod h1:516cTXxZzi4NBUBbKcwmO4Eqbb6GHAEd3o4N+GYyCBY=
@ -864,6 +879,7 @@ github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-20200501113911-9a95f0fdbf
github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-rc.2.0.20201207153454-9f6bf00c00a7 h1:guQyUpELu4I0wKgdsRBZDA5blfGiUleuppRSVy9Qbi0=
github.com/grpc-ecosystem/go-grpc-middleware/v2 v2.0.0-rc.2.0.20201207153454-9f6bf00c00a7/go.mod h1:GhphxcdlaRyAuBSvo6rV71BvQcvB/vuX8ugCyybuS2k=
github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk=
github.com/grpc-ecosystem/grpc-gateway v1.9.4/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY=
github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks=
@ -955,6 +971,9 @@ github.com/hashicorp/vault v0.10.3/go.mod h1:KfSyffbKxoVyspOdlaGVjIuwLobi07qD1bA
github.com/hashicorp/vault-plugin-secrets-kv v0.0.0-20190318174639-195e0e9d07f1/go.mod h1:VJHHT2SC1tAPrfENQeBhLlb5FbZoKZM+oC/ROmEftz0=
github.com/hashicorp/vic v1.5.1-0.20190403131502-bbfe86ec9443/go.mod h1:bEpDU35nTu0ey1EXjwNwPjI9xErAsoOCmcMb9GKvyxo=
github.com/hashicorp/yamux v0.0.0-20181012175058-2f1d1f20f75d/go.mod h1:+NfK9FKeTrX5uv1uIXGdwYDTeHna2qgaIlx54MXqjAM=
github.com/heroku/rollrus v0.2.0/go.mod h1:B3MwEcr9nmf4xj0Sr5l9eSht7wLKMa1C+9ajgAU79ek=
github.com/heroku/x v0.0.50 h1:CA0AXkSumucVJD+T+x+6c7X1iDEb+40F8GNgH5UjJwo=
github.com/heroku/x v0.0.50/go.mod h1:vr+jORZ6sG3wgEq2FAS6UbOUrz9/DxpQGN/xPHVgbSM=
github.com/hetznercloud/hcloud-go v1.33.2 h1:ptWKVYLW7YtjXzsqTFKFxwpVo3iM9UMkVPBYQE4teLU=
github.com/hetznercloud/hcloud-go v1.33.2/go.mod h1:XX/TQub3ge0yWR2yHWmnDVIrB+MQbda1pHxkUmDlUME=
github.com/hodgesds/perf-utils v0.0.8/go.mod h1:F6TfvsbtrF88i++hou29dTXlI2sfsJv+gRZDtmTJkAs=
@ -962,6 +981,7 @@ github.com/huandu/xstrings v1.3.1 h1:4jgBlKK6tLKFvO8u5pmYjG91cqytmDCDvGh7ECVFfFs
github.com/huandu/xstrings v1.3.1/go.mod h1:y5/lhBue+AyNmUVz9RLU9xbLR0o4KIIExikq4ovT0aE=
github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg=
github.com/hudl/fargo v1.4.0/go.mod h1:9Ai6uvFy5fQNq6VPKtg+Ceq1+eTY4nKUlR2JElEOcDo=
github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE=
github.com/iancoleman/strcase v0.2.0/go.mod h1:iwCmte+B7n89clKwxIoIXy/HfoL7AsD47ZCWhYzw7ho=
github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc=
github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc=
@ -1009,6 +1029,7 @@ github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9Y
github.com/jmespath/go-jmespath v0.4.0/go.mod h1:T8mJZnbsbmF+m6zOOFylbeCJqk5+pHWvzYPziyZiYoo=
github.com/jmespath/go-jmespath/internal/testify v1.5.1 h1:shLQSRRSCCPj3f2gpwzGwWFoC7ycTf1rcQZHOlsJ6N8=
github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfCI6z80xFu9LTZmf1ZRjMHUOPmWr69U=
github.com/joeshaw/envdecode v0.0.0-20180129163420-d5f34bca07f3/go.mod h1:Q+alOFAXgW5SrcfMPt/G4B2oN+qEcQRJjkn/f4mKL04=
github.com/joeshaw/multierror v0.0.0-20140124173710-69b34d4ec901/go.mod h1:Z86h9688Y0wesXCyonoVr47MasHilkuLMqGhRZ4Hpak=
github.com/joho/godotenv v1.3.0/go.mod h1:7hK45KPybAkOC6peb+G5yklZfMxEjkZhHbwpqxOKXbg=
github.com/jonboulle/clockwork v0.1.0/go.mod h1:Ii8DK3G1RaLaWxj9trq07+26W01tbo22gdxWY5EU2bo=
@ -1048,6 +1069,7 @@ github.com/karrick/godirwalk v1.10.3/go.mod h1:RoGL9dQei4vP9ilrpETWE8CLOZ1kiN0Lh
github.com/karrick/godirwalk v1.16.1/go.mod h1:j4mkqPuvaLI8mp1DroR3P6ad7cyYd4c1qeJ3RV7ULlk=
github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8=
github.com/keybase/go-crypto v0.0.0-20180614160407-5114a9a81e1b/go.mod h1:ghbZscTyKdM07+Fw3KSi0hcJm+AlEUWj8QLlPtijN/M=
github.com/keybase/go-ps v0.0.0-20190827175125-91aafc93ba19/go.mod h1:hY+WOq6m2FpbvyrI93sMaypsttvaIL5nhVR92dTMUcQ=
github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=
@ -1087,6 +1109,7 @@ github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+
github.com/lann/builder v0.0.0-20180802200727-47ae307949d0/go.mod h1:dXGbAdH5GtBTC4WfIxhKZfyBF/HBFgRZSWwZ9g/He9o=
github.com/lann/ps v0.0.0-20150810152359-62de8c46ede0/go.mod h1:vmVJ0l/dxyfGW6FmdpVm2joNMFikkuWg0EoCKLGUMNw=
github.com/leanovate/gopter v0.2.4/go.mod h1:gNcbPWNEWRe4lm+bycKqxUYoH5uoVje5SkOJ3uoLer8=
github.com/leesper/go_rng v0.0.0-20171009123644-5344a9259b21/go.mod h1:N0SVk0uhy+E1PZ3C9ctsPRlvOPAFPkCNlcPBDkt0N3U=
github.com/leesper/go_rng v0.0.0-20190531154944-a612b043e353/go.mod h1:N0SVk0uhy+E1PZ3C9ctsPRlvOPAFPkCNlcPBDkt0N3U=
github.com/leodido/go-urn v1.1.0/go.mod h1:+cyI34gQWZcE1eQU7NVgKkkzdXDQHr1dBMtdAPozLkw=
github.com/leodido/go-urn v1.2.0/go.mod h1:+8+nEpDfqqsY+g338gtMEUOtuK+4dEMhiQEgxpxOKII=
@ -1100,6 +1123,7 @@ github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0U
github.com/linode/linodego v1.6.0 h1:y3KgXttj0v6V3HyGtsvdkTl0gIzaAAOdrDXCIwGeh2g=
github.com/linode/linodego v1.6.0/go.mod h1:9lmhBsOupR6ke7D9Ioj1bq/ny9pfgFkCLiX7ubq0r08=
github.com/lovoo/gcloud-opentracing v0.3.0/go.mod h1:ZFqk2y38kMDDikZPAK7ynTTGuyt17nSPdS3K5e+ZTBY=
github.com/lstoll/grpce v1.7.0/go.mod h1:XiCWl3R+avNCT7KsTjv3qCblgsSqd0SC4ymySrH226g=
github.com/lufia/iostat v1.1.0/go.mod h1:rEPNA0xXgjHQjuI5Cy05sLlS2oRcSlWHRLrvh/AQ+Pg=
github.com/lyft/protoc-gen-star v0.6.0/go.mod h1:TGAoBVkt8w7MPG72TrKIu85MIdXwDuzJYeZuUPFPNwA=
github.com/lyft/protoc-gen-validate v0.0.0-20180911180927-64fcb82c878e/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ=
@ -1322,6 +1346,7 @@ github.com/pierrec/lz4/v4 v4.1.12 h1:44l88ehTZAUGW4VlO1QC4zkilL99M6Y9MXNwEs0uzP8
github.com/pierrec/lz4/v4 v4.1.12/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4=
github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.8.2-0.20190227000051-27936f6d90f9/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4=
github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pkg/profile v1.2.1/go.mod h1:hJw3o1OdXxsrSjjVksARp5W95eeEaEfptyVZyv6JUPA=
@ -1397,6 +1422,8 @@ github.com/prometheus/procfs v0.7.3 h1:4jVXhlkAyzOScmCkXBTOLRLTz8EeU+eyjrwB/EPq0
github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA=
github.com/prometheus/prometheus v0.36.2-0.20220613200027-59727ab0eb48 h1:b0KPmtNBsvMIfW6fryLG8yRSj3Ye3uHdQEyoTJmO/wM=
github.com/prometheus/prometheus v0.36.2-0.20220613200027-59727ab0eb48/go.mod h1:evpqrqffGRI38M1zH3IHpmXTeho8IfX5Qpx6Ixpqhyk=
github.com/rafaeljusto/redigomock v0.0.0-20190202135759-257e089e14a1/go.mod h1:JaY6n2sDr+z2WTsXkOmNRUfDy6FN0L6Nk7x06ndm4tY=
github.com/rcrowley/go-metrics v0.0.0-20160613154715-cfa5a85e9f0a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
@ -1408,6 +1435,8 @@ github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6L
github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
github.com/rollbar/rollbar-go v1.0.2/go.mod h1:AcFs5f0I+c71bpHlXNNDbOWJiKwjFDtISeXco0L5PKQ=
github.com/rollbar/rollbar-go v1.2.0/go.mod h1:czC86b8U4xdUH7W2C6gomi2jutLm8qK0OtrF5WMvpcc=
github.com/rs/cors v1.8.0/go.mod h1:EBwu+T5AvHOcXwvZIkQFjUN6s8Czyqw12GL/Y0tUyRM=
github.com/rs/cors v1.8.2/go.mod h1:XyqrcTp5zjWr1wsJ8PIRZssZ8b/WMcMf71DJnit4EMU=
github.com/rs/xid v1.2.1 h1:mhH9Nq+C1fY2l1XIpgxIiUOfNpRBYH1kKcr+qfKgjRc=
@ -1434,6 +1463,7 @@ github.com/sercand/kuberesolver v2.4.0+incompatible/go.mod h1:lWF3GL0xptCB/vCiJP
github.com/sergi/go-diff v1.0.0/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/shirou/gopsutil v0.0.0-20181107111621-48177ef5f880/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shirou/gopsutil v2.20.9+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
github.com/shirou/gopsutil/v3 v3.21.9/go.mod h1:YWp/H8Qs5fVmf17v7JNZzA0mPJ+mS2e9JdiUF9LlKzQ=
github.com/shirou/w32 v0.0.0-20160930032740-bb4de0191aa4/go.mod h1:qsXQc7+bwAM3Q1u/4XEfrquwF8Lw7D7y5cD8CuHnfIc=
github.com/shopspring/decimal v0.0.0-20180709203117-cd690d0c9e24/go.mod h1:M+9NzErvs504Cn4c5DxATwIqPbtswREoFCre64PpcG4=
github.com/shopspring/decimal v0.0.0-20200105231215-408a2507e114/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o=
@ -1458,6 +1488,7 @@ github.com/smartystreets/assertions v0.0.0-20180820201707-7c9eb446e3cf/go.mod h1
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM=
github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc=
github.com/smartystreets/goconvey v0.0.0-20180222194500-ef6db91d284a/go.mod h1:XDJAKZRPZ1CvBcN2aX5YOUTYGHki24fSF0Iv48Ibg0s=
github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/smartystreets/goconvey v1.6.4 h1:fv0U8FUIMPNf1L9lnHLvLhgicrIVChEkdzIKYqbNC9s=
github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA=
github.com/softlayer/softlayer-go v0.0.0-20180806151055-260589d94c7d/go.mod h1:Cw4GTlQccdRGSEf6KiMju767x0NEHE0YIVPJSaXjlsw=
@ -1466,6 +1497,7 @@ github.com/soniah/gosnmp v1.25.0/go.mod h1:8YvfZxH388NIIw2A+X5z2Oh97VcNhtmxDLt5Q
github.com/sony/gobreaker v0.4.1 h1:oMnRNZXX5j85zso6xCPRNPtmAycat+WcoKbklScLDgQ=
github.com/sony/gobreaker v0.4.1/go.mod h1:ZKptC7FHNvhBz7dN2LGjPVBz2sZJmc0/PkyDJOjmxWY=
github.com/soundcloud/go-runit v0.0.0-20150630195641-06ad41a06c4a/go.mod h1:LeFCbQYJ3KJlPs/FvPz2dy1tkpxyeNESVyCNNzRXFR0=
github.com/soveran/redisurl v0.0.0-20180322091936-eb325bc7a4b8/go.mod h1:FVJ8jbHu7QrNFs3bZEsv/L5JjearIAY9N0oXh2wk+6Y=
github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ=
github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA=
github.com/spf13/afero v1.2.2/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk=
@ -1474,6 +1506,7 @@ github.com/spf13/afero v1.6.0 h1:xoax2sJ2DT8S8xA2paPFjDCScCNeWsg75VG0DLRreiY=
github.com/spf13/afero v1.6.0/go.mod h1:Ai8FlHk4v/PARR026UzYexafAt9roJ7LcLMAmO6Z93I=
github.com/spf13/cast v1.3.1 h1:nFm6S0SMdyzrzcmThSipiEubIDy8WEXKNZ0UOgiRpng=
github.com/spf13/cast v1.3.1/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE=
github.com/spf13/cobra v0.0.2/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
github.com/spf13/cobra v0.0.3/go.mod h1:1l0Ry5zgKvJasoi3XT1TypsSe7PqH0Sj9dhYf7v3XqQ=
github.com/spf13/pflag v1.0.1/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
github.com/spf13/pflag v1.0.2/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4=
@ -1517,6 +1550,8 @@ github.com/tidwall/pretty v0.0.0-20180105212114-65a9db5fad51/go.mod h1:XNkn88O1C
github.com/tidwall/pretty v1.0.0 h1:HsD+QiTn7sK6flMKIvNmpqz1qrpP3Ps6jOKIKMooyg4=
github.com/tidwall/pretty v1.0.0/go.mod h1:XNkn88O1ChpSDQmQeStsy+sBenx6DDtFZJxhVysOjyk=
github.com/tinylib/msgp v1.1.5/go.mod h1:eQsjooMTnV42mHu917E26IogZ2930nFyBQdofk10Udg=
github.com/tklauser/go-sysconf v0.3.9/go.mod h1:11DU/5sG7UexIrp/O6g35hrWzu0JxlwQ3LSFUzyeuhs=
github.com/tklauser/numcpus v0.3.0/go.mod h1:yFGUr7TUHQRAhyqBcEg0Ge34zDBAsIvJJcyE6boqnA8=
github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
github.com/tonistiigi/fifo v0.0.0-20190226154929-a9fb20d87448 h1:hbyjqt5UnyKeOT3rFVxLxi7iTI6XqR2p4TkwEAQdUiw=
github.com/tonistiigi/fifo v0.0.0-20190226154929-a9fb20d87448/go.mod h1:Q5IRRDY+cjIaiOjTAnXN5LKQV5MPqVx5ofQn85Jy5Yw=
@ -1534,9 +1569,12 @@ github.com/ugorji/go v1.1.7 h1:/68gy2h+1mWMrwZFeD1kQialdSzAb432dtpeJ42ovdo=
github.com/ugorji/go v1.1.7/go.mod h1:kZn38zHttfInRq0xu/PH0az30d+z6vm202qpg1oXVMw=
github.com/ugorji/go/codec v1.1.7 h1:2SvQaVZ1ouYrrKKwoSk2pzd4A9evlKJb9oTL+OaLUSs=
github.com/ugorji/go/codec v1.1.7/go.mod h1:Ax+UKWsSmolVDwsd+7N3ZtXu+yMGCf907BLYF3GoBXY=
github.com/unrolled/secure v1.0.1/go.mod h1:R6rugAuzh4TQpbFAq69oqZggyBQxFRFQIewtz5z7Jsc=
github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA=
github.com/urfave/cli v1.21.0/go.mod h1:lxDj6qX9Q6lWQxIrbrT0nwecwUtRnhVZAJjJZrVUZZQ=
github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
github.com/urfave/cli/v2 v2.2.0/go.mod h1:SE9GqnLQmjVa0iPEY0f1w3ygNIYcIJ0OKPMoW2caLfQ=
github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI=
github.com/vektah/gqlparser v1.1.2/go.mod h1:1ycwN7Ij5njmMkPPAOaRFY4rET2Enx7IkVv3vaXspKw=
github.com/vishvananda/netlink v0.0.0-20171020171820-b2de5d10e38e/go.mod h1:+SR5DhBJrl6ZM7CoCKvpw5BKroDKQ+PJqOg65H/2ktk=
@ -1616,6 +1654,7 @@ go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk=
go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU=
go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8=
go.opencensus.io v0.22.1/go.mod h1:Ap50jQcDJrx6rB6VgeeFPtuPIf3wMRvRfrfYDO6+BmA=
go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.3/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
go.opencensus.io v0.22.4/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw=
@ -1624,18 +1663,38 @@ go.opencensus.io v0.23.0 h1:gqCw0LfLxScz8irSi8exQc7fyQ0fKQU/qnC/X8+V/1M=
go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.32.0 h1:mac9BKRqwaX6zxHPDe3pvmWpwuuIM0vuXv2juCnQevE=
go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.32.0/go.mod h1:5eCOqeGphOyz6TsY3ZDNjE33SM/TFAK3RGuCL2naTgY=
go.opentelemetry.io/otel v1.0.0-RC1/go.mod h1:x9tRa9HK4hSSq7jf2TKbqFbtt58/TGk0f9XiEYISI1I=
go.opentelemetry.io/otel v1.0.0-RC3/go.mod h1:Ka5j3ua8tZs4Rkq4Ex3hwgBgOchyPVq5S6P2lz//nKQ=
go.opentelemetry.io/otel v1.7.0 h1:Z2lA3Tdch0iDcrhJXDIlC94XE+bxok1F9B+4Lz/lGsM=
go.opentelemetry.io/otel v1.7.0/go.mod h1:5BdUoMIz5WEs0vt0CUEMtSSaTSHBBVwrhnz7+nrD5xk=
go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.7.0/go.mod h1:M1hVZHNxcbkAlcvrOMlpQ4YOO3Awf+4N2dxkZL3xm04=
go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.22.0/go.mod h1:gIp6+vQxqmh6Vd/mucqnsaFpOuVycQAS/BBXMKzJk0w=
go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.23.0/go.mod h1:pe9oOWRaZyapdajWCn64fnl76v3cmTEmNBgh7MkKvwE=
go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetricgrpc v0.23.0/go.mod h1:aSP5oMNaAfOYq+sRydHANZ0vBYLyZR/3lR9pru9aPLk=
go.opentelemetry.io/otel/exporters/otlp/otlpmetric/otlpmetrichttp v0.22.0/go.mod h1:FX3Hvv/Hwb4WtlrAvco4QCsfxsaqg/UoYA6WFCnT1Ro=
go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.7.0/go.mod h1:ceUgdyfNv4h4gLxHR0WNfDiiVmZFodZhZSbOLhpxqXE=
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.7.0/go.mod h1:E+/KKhwOSw8yoPxSSuUHG6vKppkvhN+S1Jc7Nib3k3o=
go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.7.0/go.mod h1:aFXT9Ng2seM9eizF+LfKiyPBGy8xIZKwhusC1gIu3hA=
go.opentelemetry.io/otel/internal/metric v0.22.0/go.mod h1:7qVuMihW/ktMonEfOvBXuh6tfMvvEyoIDgeJNRloYbQ=
go.opentelemetry.io/otel/internal/metric v0.23.0/go.mod h1:z+RPiDJe30YnCrOhFGivwBS+DU1JU/PiLKkk4re2DNY=
go.opentelemetry.io/otel/metric v0.22.0/go.mod h1:KcsUkBiYGW003DJ+ugd2aqIRIfjabD9jeOUXqsAtrq0=
go.opentelemetry.io/otel/metric v0.23.0/go.mod h1:G/Nn9InyNnIv7J6YVkQfpc0JCfKBNJaERBGw08nqmVQ=
go.opentelemetry.io/otel/metric v0.30.0 h1:Hs8eQZ8aQgs0U49diZoaS6Uaxw3+bBE3lcMUKBFIk3c=
go.opentelemetry.io/otel/metric v0.30.0/go.mod h1:/ShZ7+TS4dHzDFmfi1kSXMhMVubNoP0oIaBp70J6UXU=
go.opentelemetry.io/otel/oteltest v1.0.0-RC1/go.mod h1:+eoIG0gdEOaPNftuy1YScLr1Gb4mL/9lpDkZ0JjMRq4=
go.opentelemetry.io/otel/sdk v1.0.0-RC1/go.mod h1:kj6yPn7Pgt5ByRuwesbaWcRLA+V7BSDg3Hf8xRvsvf8=
go.opentelemetry.io/otel/sdk v1.0.0-RC3/go.mod h1:78H6hyg2fka0NYT9fqGuFLvly2yCxiBXDJAgLKo/2Us=
go.opentelemetry.io/otel/sdk v1.7.0/go.mod h1:uTEOTwaqIVuTGiJN7ii13Ibp75wJmYUDe374q6cZwUU=
go.opentelemetry.io/otel/sdk/export/metric v0.22.0/go.mod h1:a14rf2CiHSn9xjB6cHuv0HoZGl5C4w2PAgl+Lja1VzU=
go.opentelemetry.io/otel/sdk/export/metric v0.23.0/go.mod h1:SuMiREmKVRIwFKq73zvGTvwFpxb/ZAYkMfyqMoOtDqs=
go.opentelemetry.io/otel/sdk/metric v0.22.0/go.mod h1:LzkI0G0z6KhEagqmzgk3bw/dglE2Tk2OXs455UMcI0s=
go.opentelemetry.io/otel/sdk/metric v0.23.0/go.mod h1:wa0sKK13eeIFW+0OFjcC3S1i7FTRRiLAXe1kjBVbhwg=
go.opentelemetry.io/otel/trace v1.0.0-RC1/go.mod h1:86UHmyHWFEtWjfWPSbu0+d0Pf9Q6e1U+3ViBOc+NXAg=
go.opentelemetry.io/otel/trace v1.0.0-RC3/go.mod h1:VUt2TUYd8S2/ZRX09ZDFZQwn2RqfMB5MzO17jBojGxo=
go.opentelemetry.io/otel/trace v1.7.0 h1:O37Iogk1lEkMRXewVtZ1BBTVn5JEp8GrJvP92bJqC6o=
go.opentelemetry.io/otel/trace v1.7.0/go.mod h1:fzLSB9nqR2eXzxPXb2JW9IKE+ScyXA48yyE4TNvoHqU=
go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI=
go.opentelemetry.io/proto/otlp v0.9.0/go.mod h1:1vKfU9rv61e9EVGthD1zNvUbiwPcimSsOPU9brfSHJg=
go.opentelemetry.io/proto/otlp v0.16.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U=
go.starlark.net v0.0.0-20200901195727-6e684ef5eeee/go.mod h1:f0znQkUKRrkk36XxWbGjMqQM8wGv/xHBVE2qc3B5oFU=
go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
@ -1753,6 +1812,7 @@ golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73r
golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181005035420-146acd28ed58/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181108082009-03003ca0c849/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
golang.org/x/net v0.0.0-20181201002055-351d144fa1fc/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
@ -1893,12 +1953,15 @@ golang.org/x/sys v0.0.0-20190531175056-4c3a928424d2/go.mod h1:h1NjWce9XRLGQEsW7w
golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190712062909-fae7ac547cb7/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190804053845-51ab0e2deafa/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190826190057-c7b8b68b1456/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190902133755-9109b7679e13/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190922100055-0a153f010e69/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20190924154521-2837fb4f24fe/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@ -1963,14 +2026,17 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210630005230-0f9fa26af87c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210806184541-e5e7981a1069/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210816074244-15123e1e1f71/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210816183151-1e6c022a8912/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210823070655-63515b42dcdf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210831042530-f4d43177bf5e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210902050250-f475640dd07b/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210908233432-aa78b53d3365/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210917161153-d61c044b1678/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20210927094055-39ccf1dd6fa6/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211019181941-9d821ace8654/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211102061401-a2f17f7b995c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211124211545-fe61309f8881/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211210111614-af8b64212486/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
@ -2101,6 +2167,7 @@ golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df/go.mod h1:K8+ghG5WaK9qNq
golang.zx2c4.com/wireguard v0.0.20200121/go.mod h1:P2HsVp8SKwZEufsnezXZA4GRX/T49/HlU7DGuelXsU4=
golang.zx2c4.com/wireguard/wgctrl v0.0.0-20200205215550-e35592f146e4/go.mod h1:UdS9frhv65KTfwxME1xE8+rHYoFpbm36gOud1GhBe9c=
gonum.org/v1/gonum v0.0.0-20180816165407-929014505bf4/go.mod h1:Y+Yx5eoAFn32cQvJDxZx5Dpnq+c3wtXuadVZAcxbbBo=
gonum.org/v1/gonum v0.0.0-20190502212712-4a2eb0188cbc/go.mod h1:2ltnJ7xHfj0zHS40VVPYEAAMTa3ZGguvHGBSJeRWqE0=
gonum.org/v1/gonum v0.6.2/go.mod h1:9mxDZsDKxgMAuccQkewq682L+0eCu4dCN2yonUJTCLU=
gonum.org/v1/gonum v0.8.2/go.mod h1:oe/vMfY3deqTw+1EZJhuvEW2iwGF1bW9wwu7XCu0+v0=
gonum.org/v1/netlib v0.0.0-20190313105609-8cb42192e0e0/go.mod h1:wa6Ws7BG/ESfp6dHfk7C6KdzKA7wR7u/rKwOGE66zvw=
@ -2160,12 +2227,14 @@ google.golang.org/appengine v1.6.7 h1:FZR1q0exgwxzPzp/aF+VccGrSfxfPpkBqjIIEq3ru6
google.golang.org/appengine v1.6.7/go.mod h1:8WjMMxjGQR8xUklV/ARdw2HLXBOI7O7uCIDZVag1xfc=
google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc=
google.golang.org/genproto v0.0.0-20181221175505-bd9b4fb69e2f/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg=
google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190404172233-64821d5d2107/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190418145605-e7d98fc518a7/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190425155659-357c62f0e4bb/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190502173448-54afdca5d873/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE=
google.golang.org/genproto v0.0.0-20190530194941-fb225487d101/go.mod h1:z3L6/3dTEVtUr6QSP8miRzeRqwQOioJ9I66odjN4I7s=
google.golang.org/genproto v0.0.0-20190716160619-c506a9f90610/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190801165951-fa694d86fc64/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc=
google.golang.org/genproto v0.0.0-20190911173649-1774047e7e51/go.mod h1:IbNlFCBrqXvoKpeg0TB2l7cyZUmoaFKYIwrEpbDKLA8=
@ -2247,6 +2316,7 @@ google.golang.org/genproto v0.0.0-20220602131408-e326c6e8e9c8/go.mod h1:yKyY4AMR
google.golang.org/grpc v1.45.0 h1:NEpgUqV3Z+ZjkqMsxMg11IaDrXY4RY6CQukSGK0uI1M=
google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ=
google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw=
google.golang.org/grpc/examples v0.0.0-20210916203835-567da6b86340/go.mod h1:gID3PKrg7pWKntu9Ss6zTLJ0ttC0X9IHgREOCZwbCVU=
google.golang.org/grpc/examples v0.0.0-20211119005141-f45e61797429/go.mod h1:gID3PKrg7pWKntu9Ss6zTLJ0ttC0X9IHgREOCZwbCVU=
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=
@ -2267,6 +2337,7 @@ gopkg.in/airbrake/gobrake.v2 v2.0.9/go.mod h1:/h5ZAUhDkGaJfjzjKLSjv6zCL6O0LLBxU4
gopkg.in/alecthomas/kingpin.v2 v2.2.6 h1:jMFz6MfLP0/4fUyZle81rXUoxOBFi19VUFKVDOQfozc=
gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw=
gopkg.in/asn1-ber.v1 v1.0.0-20181015200546-f715ec2f112d/go.mod h1:cuepJuh7vyXfUyUwEgHQXw849cJrilpS5NeIjOWESAw=
gopkg.in/caio/go-tdigest.v2 v2.3.0/go.mod h1:HPfh/CLN8UWDMOC76lqxVeKa5E24ypoVuTj4BLMb9cU=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
@ -2290,6 +2361,7 @@ gopkg.in/go-playground/validator.v9 v9.29.1/go.mod h1:+c9/zcJMFNgbLvly1L1V+PpxWd
gopkg.in/gorethink/gorethink.v3 v3.0.5/go.mod h1:+3yIIHJUGMBK+wyPH+iN5TP+88ikFDfZdqTlK3Y9q8I=
gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc=
gopkg.in/inf.v0 v0.9.1/go.mod h1:cWUDdTG/fYaXco+Dcufb5Vnc6Gp2YChqWtbxRZE0mXw=
gopkg.in/ini.v1 v1.42.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/ini.v1 v1.57.0 h1:9unxIsFcTt4I55uWluz+UmL95q4kdJ0buvQ1ZIqVQww=
gopkg.in/ini.v1 v1.57.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k=
gopkg.in/jcmturner/aescts.v1 v1.0.1/go.mod h1:nsR8qBOg+OucoIW+WMhB3GspUQXq9XorLnQb9XtvcOo=
@ -2369,6 +2441,7 @@ modernc.org/sqlite v1.7.4/go.mod h1:xse4RHCm8Fzw0COf5SJqAyiDrVeDwAQthAS1V/woNIA=
modernc.org/tcl v1.4.1/go.mod h1:8YCvzidU9SIwkz7RZwlCWK61mhV8X9UwfkRDRp7y5e0=
rsc.io/binaryregexp v0.2.0 h1:HfqmD5MEmC0zvwBuF187nq9mdnXjXsSivRiXN7SmRkE=
rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8=
rsc.io/goversion v1.2.0/go.mod h1:Eih9y/uIBS3ulggl7KNJ09xGSLcuNaLgmvvqa07sgfo=
rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4=
rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0=
rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA=

@ -0,0 +1,27 @@
Copyright (c) 2018, Salesforce.com, Inc.
All rights reserved.
Redistribution and use in source and binary forms, with or without modification,
are permitted provided that the following conditions are met:
* Redistributions of source code must retain the above copyright notice, this
list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above copyright notice, this
list of conditions and the following disclaimer in the documentation and/or
other materials provided with the distribution.
* Neither the name of Salesforce.com nor the names of its contributors may be
used to endorse or promote products derived from this software without specific
prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND
ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED
WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR
ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES
(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES;
LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON
ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS
SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

@ -0,0 +1,47 @@
# logplex/encoding
## What's this?
A set of libraries we use to parse messages, and to also publish these same
syslog RFC5424 messages.
## How to use?
We have 2 scanners available. If you're trying to build a logplex compatible ingress,
you can use the regular scanner.
### Scanner
```go
func handler(w http.ResponseWriter, r *http.Request) {
s := NewScanner(r.Body)
for s.Scan() {
log.Printf("%+v", scanner.Message())
}
if s.Err() != nil {
log.Printf("err: %v", s.Err())
}
}
```
### DrainScanner
If the intent is to write an application which acts as a heroku drain,
then using the DrainScanner is preferrable -- primarily because it doesn't
require structured data.
```
func handler(w http.ResponseWriter, r *http.Request) {
s := NewDrainScanner(r.Body)
for s.Scan() {
log.Printf("%+v", scanner.Message())
}
if s.Err() != nil {
log.Printf("err: %v", s.Err())
}
}
```

@ -0,0 +1,126 @@
package encoding
import (
"fmt"
"io"
"strconv"
"github.com/pkg/errors"
)
// SyslogTimeFormat defines the exact time format used in our logs.
const SyslogTimeFormat = "2006-01-02T15:04:05.999999-07:00"
// FlexibleSyslogTimeFormat accepts both 'Z' and TZ notation for event time.
const FlexibleSyslogTimeFormat = "2006-01-02T15:04:05.999999Z07:00"
// HumanTimeFormat defines the human friendly time format used in CLI/UI.
const HumanTimeFormat = "2006-01-02T15:04:05.000000-07:00"
// L15Error is the message returned with an L15 error
const L15Error = "L15: Error displaying log lines. Please try again."
// ErrInvalidMessage returned when trying to encode an invalid syslog message
var ErrInvalidMessage = errors.New("invalid message")
// Encoder abstracts away how messages are written out
type Encoder interface {
Encode(msg Message) error
KeepAlive() error
}
type plainEncoder struct {
w io.Writer
}
// NewPlain creates a plain encoder. It dumps the log message directly
// without massaging it
func NewPlain(w io.Writer) Encoder {
return &plainEncoder{w}
}
// Encode writes the message as-is
func (p *plainEncoder) Encode(msg Message) error {
_, err := p.w.Write([]byte(messageToString(msg)))
return err
}
// KeepAlive sends a null byte.
func (p *plainEncoder) KeepAlive() error {
_, err := p.w.Write([]byte{0})
return err
}
// sseEncoder wraps an io.Writer and provides convenience methods for SSE
type sseEncoder struct {
w io.Writer
}
// NewSSE instantiates a new SSE encoder
func NewSSE(w io.Writer) Encoder {
return &sseEncoder{w}
}
// KeepAlive sends a blank comment.
func (s *sseEncoder) KeepAlive() error {
_, err := fmt.Fprintf(s.w, ": \n")
return err
}
// Encode assembles the message according to the SSE spec and writes it out
func (s *sseEncoder) Encode(msg Message) error {
// Use time as the base for creating an ID, since we need monotonic numbers that we can potentially do offsets from
s.id(msg.Timestamp.Unix())
s.data(msg)
s.separator()
return nil
}
func (s *sseEncoder) id(id int64) {
fmt.Fprintf(s.w, "id: %v\n", id)
}
func (s *sseEncoder) data(msg Message) {
fmt.Fprint(s.w, "data: ")
fmt.Fprint(s.w, messageToString(msg))
fmt.Fprint(s.w, "\n")
}
func (s *sseEncoder) separator() {
fmt.Fprint(s.w, "\n\n")
}
func messageToString(msg Message) string {
return msg.Timestamp.Format(HumanTimeFormat) + " " + msg.Application + "[" + msg.Process + "]: " + msg.Message
}
// Encode serializes a syslog message into their wire format ( octet-framed syslog )
// Disabling RFC 5424 compliance is the default and needed due to https://github.com/heroku/logplex/issues/204
func Encode(msg Message) ([]byte, error) {
sd := ""
if msg.RFCCompliant {
sd = "- "
}
if msg.Version == 0 {
return nil, errors.Wrap(ErrInvalidMessage, "version")
}
line := "<" + strconv.Itoa(int(msg.Priority)) + ">" + strconv.Itoa(int(msg.Version)) + " " +
msg.Timestamp.Format(SyslogTimeFormat) + " " +
stringOrNil(msg.Hostname) + " " +
stringOrNil(msg.Application) + " " +
stringOrNil(msg.Process) + " " +
stringOrNil(msg.ID) + " " +
sd +
msg.Message
return []byte(strconv.Itoa(len(line)) + " " + line), nil
}
func stringOrNil(s string) string {
if s == "" {
return "-"
}
return s
}

@ -0,0 +1,40 @@
package encoding
import (
"io"
"time"
)
// Message is a syslog message
type Message struct {
Timestamp time.Time
Hostname string
Application string
Process string
ID string
Message string
Version uint16
Priority uint8
RFCCompliant bool
}
// Size returns the message size in bytes, including the octet framing header
func (m Message) Size() (int, error) {
b, err := Encode(m)
if err != nil {
return 0, err
}
return len(b), nil
}
// WriteTo writes the message to a stream
func (m Message) WriteTo(w io.Writer) (int64, error) {
b, err := Encode(m)
if err != nil {
return 0, err
}
i, err := w.Write(b)
return int64(i), err
}

@ -0,0 +1,58 @@
package encoding
import (
"bufio"
"bytes"
"strconv"
"github.com/pkg/errors"
)
// SyslogSplitFunc splits the data based on the defined length prefix.
// format:
//nolint:lll
// 64 <190>1 2019-07-20T17:50:10.879238Z shuttle token shuttle - - 99\n65 <190>1 2019-07-20T17:50:10.879238Z shuttle token shuttle - - 100\n
// ^ frame size ^ boundary
func SyslogSplitFunc(data []byte, atEOF bool) (advance int, token []byte, err error) {
// first space gives us the frame size
sp := bytes.IndexByte(data, ' ')
if sp == -1 {
if atEOF && len(data) > 0 {
return 0, nil, errors.Wrap(ErrBadFrame, "missing frame length")
}
return 0, nil, nil
}
if sp == 0 {
return 0, nil, errors.Wrap(ErrBadFrame, "invalid frame length")
}
msgSize, err := strconv.ParseUint(string(data[0:sp]), 10, 64)
if err != nil {
return 0, nil, errors.Wrap(ErrBadFrame, "couldnt parse frame length")
}
// 1 here is the 'space' itself, used in the framing above
dataBoundary := sp + int(msgSize) + 1
if dataBoundary > len(data) {
if atEOF {
return 0, nil, errors.Wrapf(ErrBadFrame, "message boundary (%d) not respected length (%d)", dataBoundary, len(data))
}
return 0, nil, nil
}
return dataBoundary, data[sp+1 : dataBoundary], nil
}
// TruncatingSyslogSplitFunc enforces a maximum line length after parsing.
func TruncatingSyslogSplitFunc(maxLength int) bufio.SplitFunc {
return func(data []byte, atEOF bool) (advance int, token []byte, err error) {
advance, token, err = SyslogSplitFunc(data, atEOF)
if len(token) > maxLength {
token = token[0:maxLength]
}
return
}
}

@ -0,0 +1,264 @@
package encoding
import (
"bufio"
"bytes"
"io"
"regexp"
"strconv"
"time"
"github.com/pkg/errors"
)
const (
// MaxFrameLength is the maximum message size to parse
MaxFrameLength = 10240
// OptimalFrameLength is the initial buffer size for scanning
OptimalFrameLength = 1024
defaultRfcCompliance = true
)
var (
// ErrBadFrame is returned when the scanner cannot parse syslog message boundaries
ErrBadFrame = errors.New("bad frame")
// ErrInvalidStructuredData is returned when structure data has any value other than '-' (blank)
ErrInvalidStructuredData = errors.New("invalid structured data")
// ErrInvalidPriVal is returned when pri-val is not properly formatted
ErrInvalidPriVal = errors.New("invalid pri-val")
privalVersionRe = regexp.MustCompile(`<(\d+)>(\d)`)
)
// Decode converts a rfc5424 message to our model
func Decode(raw []byte, hasStructuredData bool) (Message, error) {
msg := Message{}
b := bytes.NewBuffer(raw)
priVal, err := syslogField(b)
if err != nil {
return msg, err
}
privalVersion := privalVersionRe.FindAllSubmatch(priVal, -1)
if len(privalVersion) != 1 || len(privalVersion[0]) != 3 {
return msg, ErrInvalidPriVal
}
prio, err := strconv.ParseUint(string(privalVersion[0][1]), 10, 8)
if err != nil {
return msg, err
}
msg.Priority = uint8(prio)
version, err := strconv.ParseUint(string(privalVersion[0][2]), 10, 16)
if err != nil {
return msg, err
}
msg.Version = uint16(version)
rawTime, err := syslogField(b)
if err != nil {
return msg, err
}
msg.Timestamp, err = time.Parse(FlexibleSyslogTimeFormat, string(rawTime))
if err != nil {
return msg, err
}
hostname, err := syslogField(b)
if err != nil {
return msg, err
}
msg.Hostname = string(hostname)
application, err := syslogField(b)
if err != nil {
return msg, err
}
msg.Application = string(application)
process, err := syslogField(b)
if err != nil {
return msg, err
}
msg.Process = string(process)
id, err := syslogField(b)
if err != nil {
return msg, err
}
msg.ID = string(id)
if hasStructuredData {
// trash structured data, as we don't use it ever
if err = trashStructuredData(b); err != nil {
return msg, err
}
}
msg.Message = b.String()
return msg, nil
}
// syslogScanner is a octet-frame syslog parser
type syslogScanner struct {
parser *bufio.Scanner
item Message
err error
rfcCompliant bool
}
// Scanner is the general purpose primitive for parsing message bodies coming
// from log-shuttle, logfwd, logplex and all sorts of logging components.
type Scanner interface {
Scan() bool
Err() error
Message() Message
}
type ScannerOption func(*syslogScanner)
func WithBuffer(optimalFrameLength, maxFrameLength int) ScannerOption {
return func(s *syslogScanner) {
s.parser.Buffer(make([]byte, optimalFrameLength), maxFrameLength)
}
}
func WithSplit(splitFunc bufio.SplitFunc) ScannerOption {
return func(s *syslogScanner) {
s.parser.Split(splitFunc)
}
}
func RFCCompliant(compliant bool) ScannerOption {
return func(s *syslogScanner) {
s.rfcCompliant = compliant
}
}
// NewScanner is a syslog octet frame stream parser
func NewScanner(r io.Reader, opts ...ScannerOption) Scanner {
s := &syslogScanner{
parser: bufio.NewScanner(r),
}
// ensure some defaults are set
s.rfcCompliant = defaultRfcCompliance
s.parser.Buffer(make([]byte, OptimalFrameLength), MaxFrameLength)
s.parser.Split(SyslogSplitFunc)
// allow customization of Buffer and Split
for _, opt := range opts {
opt(s)
}
return s
}
// Message returns the current message
func (s *syslogScanner) Message() Message {
return s.item
}
// Err returns the last scanner error
func (s *syslogScanner) Err() error {
if err := s.parser.Err(); err != nil {
return err
}
return s.err
}
// Scan returns true until all messages are parsed or an error occurs.
// When an error occur, the underlying error will be presented as `Err()`
func (s *syslogScanner) Scan() bool {
if !s.parser.Scan() {
return false
}
s.item, s.err = Decode(s.parser.Bytes(), s.rfcCompliant)
return s.err == nil
}
// NewDrainScanner returns a scanner for use with drain endpoints. The primary
// difference is that it's loose and doesn't check for structured data.
func NewDrainScanner(r io.Reader, opts ...ScannerOption) Scanner {
opts = append(opts, RFCCompliant(false))
return NewScanner(r, opts...)
}
func syslogField(b *bytes.Buffer) ([]byte, error) {
g, err := b.ReadBytes(' ')
if err != nil {
return nil, err
}
if len(g) > 0 {
g = g[:len(g)-1]
}
return g, nil
}
func trashStructuredData(b *bytes.Buffer) error {
// notice the quoting
// [meta sequenceId=\"518\"][meta somethingElse=\"bl\]ah\"]
firstChar, err := b.ReadByte()
if err != nil {
return err
}
if firstChar == '-' {
// trash the following space too
_, err = b.ReadByte()
return err
}
if firstChar != '[' {
return ErrInvalidStructuredData
}
quoting := false
bracketing := true
for {
c, err := b.ReadByte()
if err != nil {
return err
}
if !bracketing {
if c == ' ' {
// we done!
// consumed the last ']' and hit a space
break
}
if c != '[' {
return ErrInvalidStructuredData
}
bracketing = true
continue
}
// makes sure we dont catch '\]' as per RFC
// PARAM-VALUE = UTF-8-STRING ; characters '"', '\' and ']' MUST be escaped.
if quoting {
quoting = false
continue
}
switch c {
case '\\':
quoting = true
case ']':
bracketing = false
}
}
return nil
}

@ -620,6 +620,9 @@ github.com/hashicorp/memberlist
# github.com/hashicorp/serf v0.9.6
## explicit; go 1.12
github.com/hashicorp/serf/coordinate
# github.com/heroku/x v0.0.50
## explicit; go 1.12
github.com/heroku/x/logplex/encoding
# github.com/hpcloud/tail v1.0.0 => github.com/grafana/tail v0.0.0-20220426200921-98e8eb28ea4c
## explicit; go 1.13
github.com/hpcloud/tail

Loading…
Cancel
Save