diff --git a/example/docker-compose/distributed/tempo-distributed.yaml b/example/docker-compose/distributed/tempo-distributed.yaml index f4cecfacba1..94490def579 100644 --- a/example/docker-compose/distributed/tempo-distributed.yaml +++ b/example/docker-compose/distributed/tempo-distributed.yaml @@ -48,10 +48,11 @@ querier: frontend_address: query-frontend:9095 metrics_generator: - remote_write: - enabled: true - client: - url: http://prometheus:9090/api/v1/write + storage: + path: /tmp/tempo/wal + remote_write: + - url: http://prometheus:9090/api/v1/write + send_exemplars: true storage: trace: diff --git a/go.mod b/go.mod index d9107c024b1..7de0a5efaab 100644 --- a/go.mod +++ b/go.mod @@ -171,6 +171,7 @@ require ( github.com/mostynb/go-grpc-compression v1.1.15 // indirect github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f // indirect github.com/oklog/run v1.1.0 // indirect + github.com/oklog/ulid v1.3.1 // indirect github.com/open-telemetry/opentelemetry-collector-contrib/exporter/kafkaexporter v0.41.0 // indirect github.com/open-telemetry/opentelemetry-collector-contrib/internal/coreinternal v0.41.0 // indirect github.com/open-telemetry/opentelemetry-collector-contrib/internal/sharedcomponent v0.41.0 // indirect diff --git a/integration/e2e/config-metrics-generator.yaml b/integration/e2e/config-metrics-generator.yaml new file mode 100644 index 00000000000..d9fec0e64a6 --- /dev/null +++ b/integration/e2e/config-metrics-generator.yaml @@ -0,0 +1,45 @@ +metrics_generator_enabled: true + +server: + http_listen_port: 3200 + +distributor: + receivers: + jaeger: + protocols: + grpc: + log_received_traces: true + +ingester: + lifecycler: + ring: + replication_factor: 1 + +metrics_generator: + collection_interval: 1s + storage: + path: /var/tempo-metrics-generator + remote_write: + - url: http://tempo_e2e-prometheus:9090/api/v1/write + send_exemplars: true + processor: + service_graphs: + histogram_buckets: [1, 2] # seconds + span_metrics: + histogram_buckets: [1, 2] # seconds + +storage: + trace: + backend: local + local: + path: /var/tempo + +memberlist: + bind_port: 7946 + join_members: + - tempo_e2e-distributor:7946 + - tempo_e2e-ingester-1:7946 + - tempo_e2e-metrics-generator:7946 + +overrides: + metrics_generator_processors: [service-graphs, span-metrics] \ No newline at end of file diff --git a/integration/e2e/metrics_generator_test.go b/integration/e2e/metrics_generator_test.go new file mode 100644 index 00000000000..bd5ce8e53cc --- /dev/null +++ b/integration/e2e/metrics_generator_test.go @@ -0,0 +1,225 @@ +package e2e + +import ( + "context" + "fmt" + "io" + "math/rand" + "net/http" + "net/url" + "strings" + "testing" + "time" + + "github.com/grafana/e2e" + thrift "github.com/jaegertracing/jaeger/thrift-gen/jaeger" + io_prometheus_client "github.com/prometheus/client_model/go" + "github.com/prometheus/common/expfmt" + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + util "github.com/grafana/tempo/integration" +) + +const ( + configMetricsGenerator = "config-metrics-generator.yaml" + prometheusImage = "prom/prometheus:latest" +) + +func TestMetricsGenerator(t *testing.T) { + s, err := e2e.NewScenario("tempo_e2e") + require.NoError(t, err) + defer s.Close() + + require.NoError(t, util.CopyFileToSharedDir(s, configMetricsGenerator, "config.yaml")) + tempoDistributor := util.NewTempoDistributor() + tempoIngester := util.NewTempoIngester(1) + tempoMetricsGenerator := util.NewTempoMetricsGenerator() + prometheus := newPrometheus() + require.NoError(t, s.StartAndWaitReady(tempoDistributor, tempoIngester, tempoMetricsGenerator, prometheus)) + + // Wait until ingester and metrics-generator are active + isServiceActiveMatcher := func(service string) []*labels.Matcher { + return []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, "name", service), + labels.MustNewMatcher(labels.MatchEqual, "state", "ACTIVE"), + } + } + require.NoError(t, tempoDistributor.WaitSumMetricsWithOptions(e2e.Equals(1), []string{`cortex_ring_members`}, e2e.WithLabelMatchers(isServiceActiveMatcher("ingester")...), e2e.WaitMissingMetrics)) + require.NoError(t, tempoDistributor.WaitSumMetricsWithOptions(e2e.Equals(1), []string{`cortex_ring_members`}, e2e.WithLabelMatchers(isServiceActiveMatcher("metrics-generator")...), e2e.WaitMissingMetrics)) + + // Get port for the Jaeger gRPC receiver endpoint + c, err := util.NewJaegerGRPCClient(tempoDistributor.Endpoint(14250)) + require.NoError(t, err) + require.NotNil(t, c) + + // Send two spans that have a client-server relationship + r := rand.New(rand.NewSource(time.Now().UnixMilli())) + traceIDLow := r.Int63() + traceIDHigh := r.Int63() + parentSpanID := r.Int63() + + err = c.EmitBatch(context.Background(), &thrift.Batch{ + Process: &thrift.Process{ServiceName: "lb"}, + Spans: []*thrift.Span{ + { + TraceIdLow: traceIDLow, + TraceIdHigh: traceIDHigh, + SpanId: parentSpanID, + ParentSpanId: 0, + OperationName: "lb-get", + StartTime: time.Now().UnixMicro(), + Duration: int64(2 * time.Second / time.Microsecond), + Tags: []*thrift.Tag{{Key: "span.kind", VStr: stringPtr("client")}}, + }, + }, + }) + require.NoError(t, err) + + err = c.EmitBatch(context.Background(), &thrift.Batch{ + Process: &thrift.Process{ServiceName: "app"}, + Spans: []*thrift.Span{ + { + TraceIdLow: traceIDLow, + TraceIdHigh: traceIDHigh, + SpanId: r.Int63(), + ParentSpanId: parentSpanID, + OperationName: "app-handle", + StartTime: time.Now().UnixMicro(), + Duration: int64(1 * time.Second / time.Microsecond), + Tags: []*thrift.Tag{{Key: "span.kind", VStr: stringPtr("server")}}, + }, + }, + }) + require.NoError(t, err) + + // Fetch metrics from Prometheus once they are received + var metricFamilies map[string]*io_prometheus_client.MetricFamily + for { + metricFamilies, err = extractMetricsFromPrometheus(prometheus, `{__name__=~"traces_.+"}`) + require.NoError(t, err) + if len(metricFamilies) > 0 { + break + } + time.Sleep(time.Second) + } + + // Print collected metrics for easier debugging + fmt.Println() + for key, family := range metricFamilies { + fmt.Println(key) + for _, metric := range family.Metric { + fmt.Println(metric) + } + } + fmt.Println() + + // Service graphs + lbls := []string{"client", "lb", "server", "app"} + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_total", lbls)) + + assert.Equal(t, 0.0, sumValues(metricFamilies, "traces_service_graph_request_client_seconds_bucket", append(lbls, "le", "1"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_client_seconds_bucket", append(lbls, "le", "2"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_client_seconds_bucket", append(lbls, "le", "+Inf"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_client_seconds_count", lbls)) + assert.Equal(t, 2.0, sumValues(metricFamilies, "traces_service_graph_request_client_seconds_sum", lbls)) + + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_server_seconds_bucket", append(lbls, "le", "1"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_server_seconds_bucket", append(lbls, "le", "2"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_server_seconds_bucket", append(lbls, "le", "+Inf"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_server_seconds_count", lbls)) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_service_graph_request_server_seconds_sum", lbls)) + + assert.Equal(t, 0.0, sumValues(metricFamilies, "traces_service_graph_request_failed_total", nil)) + assert.Equal(t, 0.0, sumValues(metricFamilies, "traces_service_graph_unpaired_spans_total", nil)) + assert.Equal(t, 0.0, sumValues(metricFamilies, "traces_service_graph_dropped_spans_total", nil)) + + // Span metrics + lbls = []string{"service", "lb", "span_name", "lb-get", "span_kind", "SPAN_KIND_CLIENT", "span_status", "STATUS_CODE_UNSET"} + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_calls_total", lbls)) + assert.Equal(t, 0.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "1"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "2"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "+Inf"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_count", lbls)) + assert.Equal(t, 2.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_sum", lbls)) + + lbls = []string{"service", "app", "span_name", "app-handle", "span_kind", "SPAN_KIND_SERVER", "span_status", "STATUS_CODE_UNSET"} + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_calls_total", lbls)) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "1"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "2"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_bucket", append(lbls, "le", "+Inf"))) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_count", lbls)) + assert.Equal(t, 1.0, sumValues(metricFamilies, "traces_spanmetrics_duration_seconds_sum", lbls)) +} + +func newPrometheus() *e2e.HTTPService { + return e2e.NewHTTPService( + "prometheus", + prometheusImage, + e2e.NewCommandWithoutEntrypoint("/bin/prometheus", "--config.file=/etc/prometheus/prometheus.yml", "--web.enable-remote-write-receiver"), + e2e.NewHTTPReadinessProbe(9090, "/-/ready", 200, 299), + 9090, + ) +} + +// extractMetricsFromPrometheus extracts metrics stored in Prometheus using the /federate endpoint. +func extractMetricsFromPrometheus(prometheus *e2e.HTTPService, matcher string) (map[string]*io_prometheus_client.MetricFamily, error) { + url := fmt.Sprintf("http://%s/federate?match[]=%s", prometheus.HTTPEndpoint(), url.QueryEscape(matcher)) + + res, err := http.Get(url) + if err != nil { + return nil, err + } + defer res.Body.Close() + + if res.StatusCode < 200 || res.StatusCode >= 300 { + return nil, fmt.Errorf("unexpected status code %d while fetching federate metrics", res.StatusCode) + } + + body, err := io.ReadAll(res.Body) + if err != nil { + return nil, err + } + + var tp expfmt.TextParser + return tp.TextToMetricFamilies(strings.NewReader(string(body))) +} + +// sumValues calculates the sum of all metrics in the metricFamily that contain the given labels. +// filterLabels must be key-value pairs. +func sumValues(metricFamily map[string]*io_prometheus_client.MetricFamily, metric string, filterLabels []string) float64 { + if len(filterLabels)%2 != 0 { + panic(fmt.Sprintf("filterLabels must be pairs: %v", filterLabels)) + } + filterLabelsMap := map[string]string{} + for i := 0; i < len(filterLabels); i += 2 { + filterLabelsMap[filterLabels[i]] = filterLabels[i+1] + } + + sum := 0.0 + +outer: + for _, metric := range metricFamily[metric].GetMetric() { + labelMap := map[string]string{} + for _, label := range metric.GetLabel() { + labelMap[label.GetName()] = label.GetValue() + } + + for key, expectedValue := range filterLabelsMap { + value, ok := labelMap[key] + if !ok || value != expectedValue { + continue outer + } + } + + // since we fetch metrics using /federate they are all untyped + sum += metric.GetUntyped().GetValue() + } + + return sum +} + +func stringPtr(s string) *string { + return &s +} diff --git a/integration/util.go b/integration/util.go index 43c5faa87ec..378d8212337 100644 --- a/integration/util.go +++ b/integration/util.go @@ -102,6 +102,23 @@ func NewTempoIngester(replica int) *e2e.HTTPService { return s } +func NewTempoMetricsGenerator() *e2e.HTTPService { + args := []string{"-config.file=" + filepath.Join(e2e.ContainerSharedDir, "config.yaml"), "-target=metrics-generator"} + args = buildArgsWithExtra(args) + + s := e2e.NewHTTPService( + "metrics-generator", + image, + e2e.NewCommandWithoutEntrypoint("/tempo", args...), + e2e.NewHTTPReadinessProbe(3200, "/ready", 200, 299), + 3200, + ) + + s.SetBackoff(TempoBackoff()) + + return s +} + func NewTempoQueryFrontend() *e2e.HTTPService { args := []string{"-config.file=" + filepath.Join(e2e.ContainerSharedDir, "config.yaml"), "-target=query-frontend"} args = buildArgsWithExtra(args) diff --git a/modules/generator/config.go b/modules/generator/config.go index 4df97116a82..1053932516d 100644 --- a/modules/generator/config.go +++ b/modules/generator/config.go @@ -6,7 +6,7 @@ import ( "github.com/grafana/tempo/modules/generator/processor/servicegraphs" "github.com/grafana/tempo/modules/generator/processor/spanmetrics" - "github.com/grafana/tempo/modules/generator/remotewrite" + "github.com/grafana/tempo/modules/generator/storage" ) const ( @@ -34,7 +34,7 @@ type Config struct { // instances of the metrics-generator as each instance will push the same time series. AddInstanceIDLabel bool `yaml:"add_instance_id_label"` - RemoteWrite remotewrite.Config `yaml:"remote_write,omitempty"` + Storage storage.Config `yaml:"storage"` } // RegisterFlagsAndApplyDefaults registers the flags. @@ -45,7 +45,7 @@ func (cfg *Config) RegisterFlagsAndApplyDefaults(prefix string, f *flag.FlagSet) cfg.CollectionInterval = 15 * time.Second cfg.AddInstanceIDLabel = true - cfg.RemoteWrite.RegisterFlagsAndApplyDefaults(prefix, f) + cfg.Storage.RegisterFlagsAndApplyDefaults(prefix, f) } type ProcessorConfig struct { diff --git a/modules/generator/generator.go b/modules/generator/generator.go index 92c617f9581..00afc0c2995 100644 --- a/modules/generator/generator.go +++ b/modules/generator/generator.go @@ -13,11 +13,10 @@ import ( "github.com/grafana/dskit/services" "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/storage" "github.com/weaveworks/common/user" "go.uber.org/atomic" - "github.com/grafana/tempo/modules/generator/remotewrite" + "github.com/grafana/tempo/modules/generator/storage" "github.com/grafana/tempo/pkg/tempopb" "github.com/grafana/tempo/pkg/util/log" ) @@ -34,8 +33,6 @@ const ( var ErrReadOnly = errors.New("metrics-generator is shutting down") -type AppendableFactory func(userID string) storage.Appendable - type Generator struct { services.Service @@ -48,20 +45,20 @@ type Generator struct { instancesMtx sync.RWMutex instances map[string]*instance - appendableFactory AppendableFactory - subservices *services.Manager subservicesWatcher *services.FailureWatcher // When set to true, the generator will refuse incoming pushes // and will flush any remaining metrics. readOnly atomic.Bool + + reg prometheus.Registerer } // New makes a new Generator. func New(cfg *Config, overrides metricsGeneratorOverrides, reg prometheus.Registerer) (*Generator, error) { - if cfg.RemoteWrite.Enabled && cfg.RemoteWrite.Client.URL == nil { - return nil, errors.New("remote-write enabled but client URL is not configured") + if cfg.Storage.Path == "" { + return nil, errors.New("must configure metrics_generator.storage.path") } if cfg.AddInstanceIDLabel { @@ -76,6 +73,8 @@ func New(cfg *Config, overrides metricsGeneratorOverrides, reg prometheus.Regist overrides: overrides, instances: map[string]*instance{}, + + reg: reg, } // Lifecycler and ring @@ -111,12 +110,6 @@ func New(cfg *Config, overrides metricsGeneratorOverrides, reg prometheus.Regist return nil, fmt.Errorf("create ring client: %w", err) } - // Remote write - remoteWriteMetrics := remotewrite.NewMetrics(reg) - g.appendableFactory = func(userID string) storage.Appendable { - return remotewrite.NewAppendable(&cfg.RemoteWrite, log.Logger, userID, remoteWriteMetrics) - } - g.Service = services.NewBasicService(g.starting, g.running, g.stopping) return g, nil } @@ -179,14 +172,18 @@ func (g *Generator) stopping(_ error) error { } } - // Wait for generator to stop subservices, then shutdown instances and flush metrics - for id, instance := range g.instances { - err := instance.shutdown(context.Background()) - if err != nil { - level.Warn(log.Logger).Log("msg", "shutdown completed with errors", "instanceID", id, "err", err) - } + var wg sync.WaitGroup + wg.Add(len(g.instances)) + + for _, inst := range g.instances { + go func(inst *instance) { + inst.shutdown(context.Background()) + wg.Done() + }(inst) } + wg.Wait() + return nil } @@ -227,15 +224,17 @@ func (g *Generator) getOrCreateInstance(instanceID string) (*instance, error) { g.instancesMtx.Lock() defer g.instancesMtx.Unlock() + inst, ok = g.instances[instanceID] - if !ok { - var err error - inst, err = newInstance(g.cfg, instanceID, g.overrides, g.appendableFactory(instanceID)) - if err != nil { - return nil, err - } - g.instances[instanceID] = inst + if ok { + return inst, nil + } + + inst, err := g.createInstance(instanceID) + if err != nil { + return nil, err } + g.instances[instanceID] = inst return inst, nil } @@ -247,6 +246,15 @@ func (g *Generator) getInstanceByID(id string) (*instance, bool) { return inst, ok } +func (g *Generator) createInstance(id string) (*instance, error) { + wal, err := storage.New(&g.cfg.Storage, id, g.reg, log.Logger) + if err != nil { + return nil, err + } + + return newInstance(g.cfg, id, g.overrides, wal) +} + func (g *Generator) collectMetrics() { ctx, cancel := context.WithTimeout(context.Background(), g.cfg.CollectionInterval) defer cancel() @@ -255,9 +263,9 @@ func (g *Generator) collectMetrics() { defer span.Finish() for _, instance := range g.instances { - err := instance.collectAndPushMetrics(ctx) + err := instance.collectMetrics(ctx) if err != nil { - level.Error(log.Logger).Log("msg", "collecting and pushing metrics failed", "tenant", instance.instanceID, "err", err) + level.Error(log.Logger).Log("msg", "collecting metrics failed", "tenant", instance.instanceID, "err", err) } } } diff --git a/modules/generator/generator_test.go b/modules/generator/generator_test.go deleted file mode 100644 index 5983021b3db..00000000000 --- a/modules/generator/generator_test.go +++ /dev/null @@ -1,217 +0,0 @@ -package generator - -import ( - "context" - "flag" - "fmt" - "net/http" - "net/http/httptest" - "net/url" - "os" - "testing" - "time" - - gokitlog "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/grafana/dskit/flagext" - "github.com/grafana/dskit/kv/consul" - "github.com/grafana/dskit/ring" - "github.com/grafana/dskit/services" - "github.com/prometheus/client_golang/prometheus" - prometheus_common_config "github.com/prometheus/common/config" - "github.com/prometheus/common/model" - prometheus_config "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/storage/remote" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/weaveworks/common/user" - - "github.com/grafana/tempo/modules/overrides" - "github.com/grafana/tempo/pkg/tempopb" - v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/pkg/util/log" - "github.com/grafana/tempo/pkg/util/test" -) - -const localhost = "localhost" - -type metric struct { - name string - val float64 -} - -func TestGenerator(t *testing.T) { - // logs will be useful to debug problems - // TODO pass the logger as a parameter to generator.New instead of overriding a global variable - log.Logger = gokitlog.NewLogfmtLogger(gokitlog.NewSyncWriter(os.Stdout)) - - rwServer, doneCh := remoteWriteServer(t, expectedMetrics) - defer rwServer.Close() - - cfg := &Config{} - cfg.RegisterFlagsAndApplyDefaults("", &flag.FlagSet{}) - - // Ring - mockStore, _ := consul.NewInMemoryClient(ring.GetCodec(), gokitlog.NewNopLogger(), nil) - - cfg.Ring.KVStore.Mock = mockStore - cfg.Ring.ListenPort = 0 - cfg.Ring.InstanceID = localhost - cfg.Ring.InstanceAddr = localhost - - // Overrides - limitsTestConfig := defaultLimitsTestConfig() - limitsTestConfig.MetricsGeneratorProcessors = map[string]struct{}{"service-graphs": {}, "span-metrics": {}} - limits, err := overrides.NewOverrides(limitsTestConfig) - require.NoError(t, err, "unexpected error creating overrides") - - // Remote write - url, err := url.Parse(fmt.Sprintf("http://%s/receive", rwServer.Listener.Addr().String())) - require.NoError(t, err) - cfg.RemoteWrite.Enabled = true - cfg.RemoteWrite.Client = prometheus_config.DefaultRemoteWriteConfig - cfg.RemoteWrite.Client.URL = &prometheus_common_config.URL{URL: url} - - generator, err := New(cfg, limits, prometheus.NewRegistry()) - require.NoError(t, err, "unexpected error creating generator") - - err = generator.starting(context.Background()) - require.NoError(t, err, "unexpected error starting generator") - - // Send some spans - req := test.MakeBatch(10, nil) - ctx := user.InjectOrgID(context.Background(), util.FakeTenantID) - _, err = generator.PushSpans(ctx, &tempopb.PushSpansRequest{Batches: []*v1.ResourceSpans{req}}) - require.NoError(t, err, "unexpected error pushing spans") - - generator.collectMetrics() - - select { - case <-doneCh: - case <-time.After(time.Second * 5): - t.Fatal("timeout while waiting for remote write server to receive spans") - } -} - -func TestGenerator_shutdown(t *testing.T) { - // logs will be useful to debug problems - // TODO pass the logger as a parameter to generator.New instead of overriding a global variable - log.Logger = gokitlog.NewLogfmtLogger(gokitlog.NewSyncWriter(os.Stdout)) - - rwServer, doneCh := remoteWriteServer(t, expectedMetrics) - defer rwServer.Close() - - cfg := &Config{} - cfg.RegisterFlagsAndApplyDefaults("", &flag.FlagSet{}) - - // Ring - mockStore, _ := consul.NewInMemoryClient(ring.GetCodec(), gokitlog.NewNopLogger(), nil) - - cfg.Ring.KVStore.Mock = mockStore - cfg.Ring.ListenPort = 0 - cfg.Ring.InstanceID = localhost - cfg.Ring.InstanceAddr = localhost - - // Overrides - limitsTestConfig := defaultLimitsTestConfig() - limitsTestConfig.MetricsGeneratorProcessors = map[string]struct{}{"service-graphs": {}, "span-metrics": {}} - limits, err := overrides.NewOverrides(limitsTestConfig) - require.NoError(t, err, "unexpected error creating overrides") - - // Remote write - url, err := url.Parse(fmt.Sprintf("http://%s/receive", rwServer.Listener.Addr().String())) - require.NoError(t, err) - cfg.RemoteWrite.Enabled = true - cfg.RemoteWrite.Client = prometheus_config.DefaultRemoteWriteConfig - cfg.RemoteWrite.Client.URL = &prometheus_common_config.URL{URL: url} - - // Set incredibly high collection interval - cfg.CollectionInterval = time.Hour - - generator, err := New(cfg, limits, prometheus.NewRegistry()) - require.NoError(t, err, "unexpected error creating generator") - - err = services.StartAndAwaitRunning(context.Background(), generator) - require.NoError(t, err, "unexpected error starting generator") - - // Send some spans - req := test.MakeBatch(10, nil) - ctx := user.InjectOrgID(context.Background(), util.FakeTenantID) - _, err = generator.PushSpans(ctx, &tempopb.PushSpansRequest{Batches: []*v1.ResourceSpans{req}}) - require.NoError(t, err, "unexpected error pushing spans") - - err = services.StopAndAwaitTerminated(context.Background(), generator) - require.NoError(t, err, "failed to terminate metrics-generator") - - select { - case <-doneCh: - case <-time.After(time.Second * 5): - t.Fatal("timeout while waiting for remote write server to receive spans") - } -} - -func remoteWriteServer(t *testing.T, expected []metric) (*httptest.Server, chan struct{}) { - doneCh := make(chan struct{}) - - mux := http.NewServeMux() - mux.HandleFunc("/receive", func(w http.ResponseWriter, r *http.Request) { - req, err := remote.DecodeWriteRequest(r.Body) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - level.Info(log.Logger).Log("msg", "received remote write", "body", req.String()) - - for i, ts := range req.Timeseries { - m := make(model.Metric, len(ts.Labels)) - for _, l := range ts.Labels { - m[model.LabelName(l.Name)] = model.LabelValue(l.Value) - } - - // don't bother testing timeseries with exemplars for now, this is already covered by other tests - if len(ts.Exemplars) != 0 { - continue - } - - if i >= len(expected) { - assert.Fail(t, "received unexpected metric", "%s", m.String()) - continue - } - - assert.Equal(t, expected[i].name, m.String()) - - assert.Len(t, ts.Samples, 1) - assert.Equal(t, expected[i].val, ts.Samples[0].Value) - } - close(doneCh) - }) - - return httptest.NewServer(mux), doneCh -} - -var expectedMetrics = []metric{ - {`traces_spanmetrics_calls_total{service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_count{service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_sum{service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.002", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.004", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.008", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.016", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.032", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.064", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.128", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.256", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="0.512", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 0}, - {`traces_spanmetrics_duration_seconds_bucket{le="1.024", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_bucket{le="2.048", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_bucket{le="4.096", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, - {`traces_spanmetrics_duration_seconds_bucket{le="+Inf", service="test-service", span_kind="SPAN_KIND_CLIENT", span_name="test", span_status="STATUS_CODE_OK", tempo_instance_id="localhost"}`, 10}, -} - -func defaultLimitsTestConfig() overrides.Limits { - limits := overrides.Limits{} - flagext.DefaultValues(&limits) - return limits -} diff --git a/modules/generator/instance.go b/modules/generator/instance.go index 5a6b826e54e..7ba48a1f7e7 100644 --- a/modules/generator/instance.go +++ b/modules/generator/instance.go @@ -11,12 +11,12 @@ import ( "github.com/opentracing/opentracing-go" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/prometheus/storage" "github.com/weaveworks/common/tracing" "github.com/grafana/tempo/modules/generator/processor" "github.com/grafana/tempo/modules/generator/processor/servicegraphs" "github.com/grafana/tempo/modules/generator/processor/spanmetrics" + "github.com/grafana/tempo/modules/generator/storage" "github.com/grafana/tempo/pkg/tempopb" "github.com/grafana/tempo/pkg/util/log" ) @@ -52,8 +52,8 @@ type instance struct { instanceID string overrides metricsGeneratorOverrides - registry processor.Registry - appendable storage.Appendable + registry processor.Registry + wal storage.Storage // processorsMtx protects the processors map, not the processors itself processorsMtx sync.RWMutex @@ -62,14 +62,14 @@ type instance struct { shutdownCh chan struct{} } -func newInstance(cfg *Config, instanceID string, overrides metricsGeneratorOverrides, appendable storage.Appendable) (*instance, error) { +func newInstance(cfg *Config, instanceID string, overrides metricsGeneratorOverrides, wal storage.Storage) (*instance, error) { i := &instance{ cfg: cfg, instanceID: instanceID, overrides: overrides, - registry: processor.NewRegistry(cfg.ExternalLabels, instanceID), - appendable: appendable, + registry: processor.NewRegistry(cfg.ExternalLabels, instanceID), + wal: wal, processors: make(map[string]processor.Processor), @@ -237,14 +237,14 @@ func (i *instance) updatePushMetrics(req *tempopb.PushSpansRequest) { metricSpansIngested.WithLabelValues(i.instanceID).Add(float64(spanCount)) } -func (i *instance) collectAndPushMetrics(ctx context.Context) error { - span, ctx := opentracing.StartSpanFromContext(ctx, "instance.collectAndPushMetrics") +func (i *instance) collectMetrics(ctx context.Context) error { + span, ctx := opentracing.StartSpanFromContext(ctx, "instance.collectMetrics") defer span.Finish() traceID, _ := tracing.ExtractTraceID(ctx) level.Info(log.Logger).Log("msg", "collecting metrics", "tenant", i.instanceID, "traceID", traceID) - appender := i.appendable.Appender(ctx) + appender := i.wal.Appender(ctx) err := i.registry.Gather(appender) if err != nil { @@ -256,20 +256,23 @@ func (i *instance) collectAndPushMetrics(ctx context.Context) error { // shutdown stops the instance and flushes any remaining data. After shutdown // is called pushSpans should not be called anymore. -func (i *instance) shutdown(ctx context.Context) error { +func (i *instance) shutdown(ctx context.Context) { close(i.shutdownCh) - err := i.collectAndPushMetrics(ctx) + err := i.collectMetrics(ctx) if err != nil { level.Error(log.Logger).Log("msg", "collecting metrics failed at shutdown", "tenant", i.instanceID, "err", err) } + err = i.wal.Close() + if err != nil { + level.Error(log.Logger).Log("msg", "closing wal failed", "tenant", i.instanceID, "err", err) + } + i.processorsMtx.Lock() defer i.processorsMtx.Unlock() for processorName := range i.processors { i.removeProcessor(processorName) } - - return err } diff --git a/modules/generator/instance_test.go b/modules/generator/instance_test.go index 9b8001593a8..dee7921dd1d 100644 --- a/modules/generator/instance_test.go +++ b/modules/generator/instance_test.go @@ -5,17 +5,20 @@ import ( "testing" "time" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + prometheus_storage "github.com/prometheus/prometheus/storage" "github.com/stretchr/testify/assert" "github.com/grafana/tempo/modules/generator/processor/servicegraphs" - "github.com/grafana/tempo/modules/generator/remotewrite" + "github.com/grafana/tempo/modules/generator/storage" "github.com/grafana/tempo/pkg/tempopb" v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" "github.com/grafana/tempo/pkg/util/test" ) func Test_instance_concurrency(t *testing.T) { - instance, err := newInstance(&Config{}, "test", &mockOverrides{}, &remotewrite.NoopAppender{}) + instance, err := newInstance(&Config{}, "test", &mockOverrides{}, &noopStorage{}) assert.NoError(t, err) end := make(chan struct{}) @@ -37,7 +40,7 @@ func Test_instance_concurrency(t *testing.T) { }) go accessor(func() { - err := instance.collectAndPushMetrics(context.Background()) + err := instance.collectMetrics(context.Background()) assert.NoError(t, err) }) @@ -59,8 +62,7 @@ func Test_instance_concurrency(t *testing.T) { time.Sleep(100 * time.Millisecond) - err = instance.shutdown(context.Background()) - assert.NoError(t, err) + instance.shutdown(context.Background()) time.Sleep(10 * time.Millisecond) close(end) @@ -68,12 +70,11 @@ func Test_instance_concurrency(t *testing.T) { } func Test_instance_updateProcessors(t *testing.T) { - instance, err := newInstance(&Config{}, "test", &mockOverrides{}, &remotewrite.NoopAppender{}) + instance, err := newInstance(&Config{}, "test", &mockOverrides{}, &noopStorage{}) assert.NoError(t, err) - // shutdown the instance to stop the update goroutine - err = instance.shutdown(context.Background()) - assert.NoError(t, err) + // stop the update goroutine + close(instance.shutdownCh) // no processors should be present initially assert.Len(t, instance.processors, 0) @@ -118,3 +119,35 @@ var _ metricsGeneratorOverrides = (*mockOverrides)(nil) func (m *mockOverrides) MetricsGeneratorProcessors(userID string) map[string]struct{} { return m.processors } + +type noopStorage struct{} + +var _ storage.Storage = (*noopStorage)(nil) + +func (m noopStorage) Appender(ctx context.Context) prometheus_storage.Appender { + return &noopAppender{} +} + +func (m noopStorage) Close() error { + return nil +} + +type noopAppender struct{} + +var _ prometheus_storage.Appender = (*noopAppender)(nil) + +func (n noopAppender) Append(ref prometheus_storage.SeriesRef, l labels.Labels, t int64, v float64) (prometheus_storage.SeriesRef, error) { + return 0, nil +} + +func (n noopAppender) AppendExemplar(ref prometheus_storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (prometheus_storage.SeriesRef, error) { + return 0, nil +} + +func (n noopAppender) Commit() error { + return nil +} + +func (n noopAppender) Rollback() error { + return nil +} diff --git a/modules/generator/processor/registry.go b/modules/generator/processor/registry.go index 821d9cb6642..d7bf1c349ca 100644 --- a/modules/generator/processor/registry.go +++ b/modules/generator/processor/registry.go @@ -111,7 +111,7 @@ func (r *Registry) Gather(appender storage.Appender) (err error) { } bucketWithLeLabels := copyWithLabel(bucketLabels, "le", fmt.Sprintf("%g", bucket.GetUpperBound())) - _, err = appender.Append(0, bucketWithLeLabels, timestamp, float64(bucket.GetCumulativeCount())) + ref, err := appender.Append(0, bucketWithLeLabels, timestamp, float64(bucket.GetCumulativeCount())) if err != nil { return err } @@ -119,7 +119,7 @@ func (r *Registry) Gather(appender storage.Appender) (err error) { e := bucket.GetExemplar() if e != nil { - _, err = appender.AppendExemplar(0, bucketWithLeLabels, exemplar.Exemplar{ + _, err = appender.AppendExemplar(ref, bucketWithLeLabels, exemplar.Exemplar{ Labels: labelPairsToLabels(e.GetLabel()), Value: e.GetValue(), Ts: e.GetTimestamp().AsTime().UnixMilli(), diff --git a/modules/generator/remotewrite/appendable.go b/modules/generator/remotewrite/appendable.go deleted file mode 100644 index 3d4bb36cad4..00000000000 --- a/modules/generator/remotewrite/appendable.go +++ /dev/null @@ -1,55 +0,0 @@ -package remotewrite - -import ( - "context" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/prometheus/prometheus/storage" -) - -// remoteWriteAppendable is a Prometheus storage.Appendable that remote writes samples and exemplars. -type remoteWriteAppendable struct { - logger log.Logger - tenantID string - cfg *Config - - // TODO add overrides/limits - - metrics *Metrics -} - -var _ storage.Appendable = (*remoteWriteAppendable)(nil) - -// NewAppendable creates a Prometheus storage.Appendable that can remote write. If -// tenantID is not empty, it sets the X-Scope-Orgid header on every request. -func NewAppendable(cfg *Config, logger log.Logger, tenantID string, metrics *Metrics) storage.Appendable { - if !cfg.Enabled { - level.Info(logger).Log("msg", "remote-write is disabled") - return &NoopAppender{} - } - - return &remoteWriteAppendable{ - logger: logger, - tenantID: tenantID, - cfg: cfg, - metrics: metrics, - } -} - -func (a *remoteWriteAppendable) Appender(ctx context.Context) storage.Appender { - client, err := newRemoteWriteClient(&a.cfg.Client, a.tenantID) - if err != nil { - level.Error(a.logger).Log("msg", "error creating remote-write client; setting appender as noop", "err", err, "tenant", a.tenantID) - return &NoopAppender{} - } - - return &remoteWriteAppender{ - logger: a.logger, - ctx: ctx, - remoteWriter: client, - sendExamplars: a.cfg.Client.SendExemplars, - userID: a.tenantID, - metrics: a.metrics, - } -} diff --git a/modules/generator/remotewrite/appendable_test.go b/modules/generator/remotewrite/appendable_test.go deleted file mode 100644 index 3f08009009d..00000000000 --- a/modules/generator/remotewrite/appendable_test.go +++ /dev/null @@ -1,154 +0,0 @@ -package remotewrite - -import ( - "context" - "fmt" - "net/http" - "net/http/httptest" - "net/url" - "os" - "testing" - "time" - - gokitlog "github.com/go-kit/log" - "github.com/prometheus/client_golang/prometheus" - prometheus_common_config "github.com/prometheus/common/config" - "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/prompb" - "github.com/prometheus/prometheus/storage/remote" - "github.com/stretchr/testify/assert" -) - -func Test_remoteWriteAppendable(t *testing.T) { - theTime := time.Now() - - var capturedTimeseries []prompb.TimeSeries - clientCfg, cleanup := createTimeSeriesServer(t, &capturedTimeseries) - defer cleanup() - - clientCfg.SendExemplars = true - - cfg := &Config{ - Enabled: true, - Client: clientCfg, - } - tenantID := "my-tenant" - - appendable := NewAppendable(cfg, gokitlog.NewLogfmtLogger(os.Stdout), tenantID, NewMetrics(prometheus.NewRegistry())) - - appender := appendable.Appender(context.Background()) - - _, err := appender.Append(0, labels.Labels{{Name: "label", Value: "append-before-rollback"}}, theTime.UnixMilli(), 0.1) - assert.NoError(t, err) - - // Rollback the appender, this should discard previously appended samples - err = appender.Rollback() - assert.NoError(t, err) - - err = appender.Commit() - assert.NoError(t, err) - - assert.Len(t, capturedTimeseries, 0) - - _, err = appender.Append(0, labels.Labels{{Name: "label", Value: "value"}}, theTime.UnixMilli(), 0.2) - assert.NoError(t, err) - _, err = appender.AppendExemplar(0, labels.Labels{{Name: "label", Value: "value"}}, exemplar.Exemplar{ - Labels: labels.Labels{{Name: "traceID", Value: "123"}}, - Value: 0.2, - Ts: theTime.UnixMilli(), - HasTs: true, - }) - assert.NoError(t, err) - - err = appender.Commit() - assert.NoError(t, err) - - assert.Len(t, capturedTimeseries, 2) - assert.Len(t, capturedTimeseries[0].Labels, 1) - assert.Equal(t, `name:"label" value:"value" `, capturedTimeseries[0].Labels[0].String()) - assert.Len(t, capturedTimeseries[0].Samples, 1) - assert.Equal(t, fmt.Sprintf(`value:0.2 timestamp:%d `, theTime.UnixMilli()), capturedTimeseries[0].Samples[0].String()) - assert.Len(t, capturedTimeseries[1].Exemplars, 1) -} - -func Test_remoteWriteAppendable_disabled(t *testing.T) { - server := httptest.NewServer(http.HandlerFunc(func(res http.ResponseWriter, req *http.Request) { - t.Fatal("server should never be called") - })) - defer server.Close() - - url, err := url.Parse(fmt.Sprintf("http://%s/receive", server.Listener.Addr().String())) - assert.NoError(t, err) - - clientCfg := config.DefaultRemoteWriteConfig - clientCfg.URL = &prometheus_common_config.URL{URL: url} - - cfg := &Config{ - Enabled: false, - Client: clientCfg, - } - - appendable := NewAppendable(cfg, gokitlog.NewLogfmtLogger(os.Stdout), "", NewMetrics(prometheus.NewRegistry())) - - appender := appendable.Appender(context.Background()) - - _, err = appender.Append(0, labels.Labels{{Name: "label", Value: "value"}}, time.Now().UnixMilli(), 0.1) - assert.NoError(t, err) - - err = appender.Commit() - assert.NoError(t, err) -} - -func Test_remoteWriteAppendable_dontSendExemplars(t *testing.T) { - theTime := time.Now() - - var capturedTimeseries []prompb.TimeSeries - clientCfg, cleanup := createTimeSeriesServer(t, &capturedTimeseries) - defer cleanup() - - clientCfg.SendExemplars = false - - cfg := &Config{ - Enabled: true, - Client: clientCfg, - } - tenantID := "my-tenant" - - appendable := NewAppendable(cfg, gokitlog.NewLogfmtLogger(os.Stdout), tenantID, NewMetrics(prometheus.NewRegistry())) - - appender := appendable.Appender(context.Background()) - - _, err := appender.AppendExemplar(0, labels.Labels{{Name: "label", Value: "value"}}, exemplar.Exemplar{ - Labels: labels.Labels{{Name: "traceID", Value: "123"}}, - Value: 0.2, - Ts: theTime.UnixMilli(), - HasTs: true, - }) - assert.NoError(t, err) - - err = appender.Commit() - assert.NoError(t, err) - - assert.Len(t, capturedTimeseries, 0) -} - -func createTimeSeriesServer(t *testing.T, capturedTimeseries *[]prompb.TimeSeries) (config.RemoteWriteConfig, func()) { - server := httptest.NewServer(http.HandlerFunc(func(res http.ResponseWriter, req *http.Request) { - writeRequest, err := remote.DecodeWriteRequest(req.Body) - assert.NoError(t, err) - - *capturedTimeseries = append(*capturedTimeseries, writeRequest.GetTimeseries()...) - })) - - url, err := url.Parse(fmt.Sprintf("http://%s/receive", server.Listener.Addr().String())) - assert.NoError(t, err) - - clientCfg := config.DefaultRemoteWriteConfig - clientCfg.URL = &prometheus_common_config.URL{URL: url} - - return clientCfg, func() { - server.Close() - } -} diff --git a/modules/generator/remotewrite/appender.go b/modules/generator/remotewrite/appender.go deleted file mode 100644 index 1c5db0bbf91..00000000000 --- a/modules/generator/remotewrite/appender.go +++ /dev/null @@ -1,161 +0,0 @@ -package remotewrite - -import ( - "context" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/klauspost/compress/snappy" - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/prompb" - "github.com/prometheus/prometheus/storage" -) - -// TODO: make this configurable -var maxWriteRequestSize = 3 * 1024 * 1024 // 3MB - -// remoteWriteAppender is a storage.Appender that remote writes samples and exemplars. -type remoteWriteAppender struct { - logger log.Logger - ctx context.Context - remoteWriter *remoteWriteClient - sendExamplars bool - userID string - - // TODO Loki uses util.EvictingQueue here to limit the amount of samples written per remote write request - labels [][]prompb.Label - samples []prompb.Sample - exemplarLabels [][]prompb.Label - exemplars []prompb.Exemplar - - metrics *Metrics -} - -var _ storage.Appender = (*remoteWriteAppender)(nil) - -func (a *remoteWriteAppender) Append(_ storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { - a.labels = append(a.labels, labelsToLabelsProto(l)) - a.samples = append(a.samples, prompb.Sample{ - Timestamp: t, - Value: v, - }) - return 0, nil -} - -func (a *remoteWriteAppender) AppendExemplar(_ storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { - if !a.sendExamplars { - return 0, nil - } - - a.exemplarLabels = append(a.exemplarLabels, labelsToLabelsProto(l)) - a.exemplars = append(a.exemplars, prompb.Exemplar{ - Labels: labelsToLabelsProto(e.Labels), - Value: e.Value, - Timestamp: e.Ts, - }) - return 0, nil -} - -func (a *remoteWriteAppender) Commit() error { - level.Debug(a.logger).Log("msg", "writing samples to remote_write target", "tenant", a.userID, "target", a.remoteWriter.Endpoint(), "count", len(a.samples)) - - if len(a.samples) == 0 { - return nil - } - - reqs := a.buildRequests() - - a.metrics.samplesSent.WithLabelValues(a.userID).Add(float64(len(a.samples))) - a.metrics.exemplarsSent.WithLabelValues(a.userID).Add(float64(len(a.exemplars))) - a.metrics.remoteWriteTotal.WithLabelValues(a.userID).Add(float64(len(reqs))) - - err := a.sendRequests(reqs) - if err != nil { - level.Error(a.logger).Log("msg", "error sending remote-write requests", "tenant", a.userID, "target", a.remoteWriter.Endpoint(), "err", err) - a.metrics.remoteWriteErrors.WithLabelValues(a.userID).Inc() - return err - } - - a.clearBuffers() - return nil -} - -// buildRequests builds a slice of prompb.WriteRequest of which each requests has a maximum size of -// maxWriteRequestSize (uncompressed). -func (a *remoteWriteAppender) buildRequests() []*prompb.WriteRequest { - var requests []*prompb.WriteRequest - currentRequest := &prompb.WriteRequest{} - - appendTimeSeries := func(ts prompb.TimeSeries) { - if currentRequest.Size()+ts.Size() >= maxWriteRequestSize { - requests = append(requests, currentRequest) - currentRequest = &prompb.WriteRequest{} - } - currentRequest.Timeseries = append(currentRequest.Timeseries, ts) - } - - for i, s := range a.samples { - appendTimeSeries(prompb.TimeSeries{ - Labels: a.labels[i], - Samples: []prompb.Sample{s}, - }) - } - - for i, e := range a.exemplars { - appendTimeSeries(prompb.TimeSeries{ - Labels: a.exemplarLabels[i], - Exemplars: []prompb.Exemplar{e}, - }) - } - - if len(currentRequest.Timeseries) != 0 { - requests = append(requests, currentRequest) - } - - return requests -} - -func (a *remoteWriteAppender) sendRequests(reqs []*prompb.WriteRequest) error { - for _, req := range reqs { - err := a.sendWriteRequest(req) - if err != nil { - return err - } - } - return nil -} - -func (a *remoteWriteAppender) sendWriteRequest(req *prompb.WriteRequest) error { - bytes, err := req.Marshal() - if err != nil { - return err - } - bytes = snappy.Encode(nil, bytes) - - // TODO the returned error can be of type RecoverableError with a retryAfter duration, should we do something with this? - return a.remoteWriter.Store(a.ctx, bytes) -} - -func (a *remoteWriteAppender) Rollback() error { - a.clearBuffers() - return nil -} - -func (a *remoteWriteAppender) clearBuffers() { - a.labels = nil - a.samples = nil - a.exemplars = nil - a.exemplarLabels = nil -} - -func labelsToLabelsProto(labels labels.Labels) []prompb.Label { - result := make([]prompb.Label, len(labels)) - for i, l := range labels { - result[i] = prompb.Label{ - Name: l.Name, - Value: l.Value, - } - } - return result -} diff --git a/modules/generator/remotewrite/appender_test.go b/modules/generator/remotewrite/appender_test.go deleted file mode 100644 index 136a72cdfa4..00000000000 --- a/modules/generator/remotewrite/appender_test.go +++ /dev/null @@ -1,105 +0,0 @@ -package remotewrite - -import ( - "bytes" - "context" - "fmt" - "os" - "testing" - "time" - - gokitlog "github.com/go-kit/log" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/prompb" - "github.com/prometheus/prometheus/storage/remote" - "github.com/stretchr/testify/assert" -) - -func Test_remoteWriteAppendable_splitRequests(t *testing.T) { - nowMs := time.Now().UnixMilli() - - // swap out global maxWriteRequestSize during test - originalMaxWriteRequestSize := maxWriteRequestSize - maxWriteRequestSize = 120 // roughly corresponds with 2 timeseries per request - defer func() { - maxWriteRequestSize = originalMaxWriteRequestSize - }() - - mockWriteClient := &mockWriteClient{} - - appender := &remoteWriteAppender{ - logger: gokitlog.NewLogfmtLogger(os.Stdout), - ctx: context.Background(), - remoteWriter: &remoteWriteClient{WriteClient: mockWriteClient}, - userID: "", - sendExamplars: true, - metrics: NewMetrics(prometheus.NewRegistry()), - } - - // Send samples - for i := 0; i < 3; i++ { - _, err := appender.Append(0, labels.Labels{{Name: "label", Value: "value"}}, nowMs, float64(i+1)) - assert.NoError(t, err) - } - // Send exemplars - for i := 0; i < 2; i++ { - _, err := appender.AppendExemplar(0, labels.Labels{{Name: "label", Value: "value"}}, exemplar.Exemplar{ - Labels: labels.Labels{{Name: "exemplarLabel", Value: "exemplarValue"}}, - Value: float64(i + 1), - Ts: nowMs, - HasTs: true, - }) - assert.NoError(t, err) - } - - err := appender.Commit() - assert.NoError(t, err) - - assert.Equal(t, mockWriteClient.storeInvocations, 3) - - // Verify samples - for i := 0; i < 3; i++ { - timeseries := mockWriteClient.capturedTimeseries[i] - - assert.Equal(t, `name:"label" value:"value" `, timeseries.Labels[0].String()) - assert.Equal(t, fmt.Sprintf(`value:%d timestamp:%d `, i+1, nowMs), timeseries.Samples[0].String()) - assert.Len(t, timeseries.Exemplars, 0) - } - // Verify exemplars - for i := 0; i < 2; i++ { - timeseries := mockWriteClient.capturedTimeseries[i+3] - - assert.Equal(t, `name:"label" value:"value" `, timeseries.Labels[0].String()) - assert.Len(t, timeseries.Samples, 0) - assert.Equal(t, fmt.Sprintf(`labels: value:%d timestamp:%d `, i+1, nowMs), timeseries.Exemplars[0].String()) - } -} - -type mockWriteClient struct { - storeInvocations int - capturedTimeseries []prompb.TimeSeries -} - -var _ remote.WriteClient = (*mockWriteClient)(nil) - -func (m *mockWriteClient) Name() string { - return "mockWriteClient" -} - -func (m *mockWriteClient) Endpoint() string { - return "mockEndpoint" -} - -func (m *mockWriteClient) Store(ctx context.Context, b []byte) error { - m.storeInvocations++ - - writeRequest, err := remote.DecodeWriteRequest(bytes.NewReader(b)) - if err != nil { - return err - } - m.capturedTimeseries = append(m.capturedTimeseries, writeRequest.Timeseries...) - - return nil -} diff --git a/modules/generator/remotewrite/client.go b/modules/generator/remotewrite/client.go deleted file mode 100644 index a3f489a42f4..00000000000 --- a/modules/generator/remotewrite/client.go +++ /dev/null @@ -1,59 +0,0 @@ -package remotewrite - -import ( - "fmt" - - "github.com/prometheus/prometheus/config" - "github.com/prometheus/prometheus/storage/remote" - - "github.com/grafana/tempo/cmd/tempo/build" -) - -const ( - userAgentHeader = "User-Agent" - xScopeOrgIDHeader = "X-Scope-Orgid" -) - -var remoteWriteUserAgent = fmt.Sprintf("tempo-remote-write/%s", build.Version) - -type remoteWriteClient struct { - remote.WriteClient -} - -// newRemoteWriteClient creates a Prometheus remote.WriteClient. If tenantID is not empty, it sets -// the X-Scope-Orgid header on every request. -func newRemoteWriteClient(cfg *config.RemoteWriteConfig, tenantID string) (*remoteWriteClient, error) { - headers := copyMap(cfg.Headers) - headers[userAgentHeader] = remoteWriteUserAgent - if tenantID != "" { - headers[xScopeOrgIDHeader] = tenantID - } - - writeClient, err := remote.NewWriteClient( - "metrics_generator", - &remote.ClientConfig{ - URL: cfg.URL, - Timeout: cfg.RemoteTimeout, - HTTPClientConfig: cfg.HTTPClientConfig, - Headers: headers, - }, - ) - if err != nil { - return nil, fmt.Errorf("could not create remote-write client for tenant: %s", tenantID) - } - - return &remoteWriteClient{ - WriteClient: writeClient, - }, nil -} - -// copyMap creates a new map containing all values from the given map. -func copyMap(m map[string]string) map[string]string { - newMap := make(map[string]string, len(m)) - - for k, v := range m { - newMap[k] = v - } - - return newMap -} diff --git a/modules/generator/remotewrite/client_test.go b/modules/generator/remotewrite/client_test.go deleted file mode 100644 index d83e6546140..00000000000 --- a/modules/generator/remotewrite/client_test.go +++ /dev/null @@ -1,102 +0,0 @@ -package remotewrite - -import ( - "context" - "fmt" - "io" - "net/http" - "net/http/httptest" - "net/url" - "strconv" - "testing" - - prometheus_common_config "github.com/prometheus/common/config" - "github.com/prometheus/prometheus/config" - "github.com/stretchr/testify/assert" -) - -func Test_remoteWriteClient(t *testing.T) { - var err error - var capturedHeaders http.Header - var capturedBody []byte - - server := httptest.NewServer(http.HandlerFunc(func(res http.ResponseWriter, req *http.Request) { - capturedHeaders = req.Header - capturedBody, err = io.ReadAll(req.Body) - assert.NoError(t, err) - })) - defer server.Close() - - url, err := url.Parse(fmt.Sprintf("http://%s/receive", server.Listener.Addr().String())) - assert.NoError(t, err) - - cfg := &config.DefaultRemoteWriteConfig - cfg.URL = &prometheus_common_config.URL{URL: url} - - t.Run("remoteWriteClient with custom headers", func(t *testing.T) { - cfg.Headers = map[string]string{ - // User-Agent header can not be overridden - userAgentHeader: "my-custom-user-agent", - "Authorization": "Basic *****", - } - data := []byte("test request #1") - - client, err := newRemoteWriteClient(cfg, "") - assert.NoError(t, err) - - err = client.Store(context.Background(), data) - assert.NoError(t, err) - - assert.Equal(t, data, capturedBody) - expectedHeaders := http.Header{ - "Authorization": {"Basic *****"}, - "Content-Encoding": {"snappy"}, - "Content-Length": {strconv.Itoa(len(data))}, - "Content-Type": {"application/x-protobuf"}, - userAgentHeader: {remoteWriteUserAgent}, - "X-Prometheus-Remote-Write-Version": {"0.1.0"}, - } - assert.Equal(t, expectedHeaders, capturedHeaders) - }) - - t.Run("remoteWriteClient with tenantID", func(t *testing.T) { - cfg.Headers = nil - data := []byte("test request #2") - - clientWithXScopeOrg, err := newRemoteWriteClient(cfg, "my-tenant") - assert.NoError(t, err) - - err = clientWithXScopeOrg.Store(context.Background(), data) - assert.NoError(t, err) - - assert.Equal(t, data, capturedBody) - expectedHeaders := http.Header{ - "Content-Encoding": {"snappy"}, - "Content-Length": {strconv.Itoa(len(data))}, - "Content-Type": {"application/x-protobuf"}, - userAgentHeader: {remoteWriteUserAgent}, - "X-Prometheus-Remote-Write-Version": {"0.1.0"}, - xScopeOrgIDHeader: {"my-tenant"}, - } - assert.Equal(t, expectedHeaders, capturedHeaders) - }) - -} - -func Test_copyMap(t *testing.T) { - original := map[string]string{ - "k1": "v1", - "k2": "v2", - } - - copied := copyMap(original) - - assert.Equal(t, original, copied) - - copied["k2"] = "other value" - copied["k3"] = "v3" - - assert.Len(t, original, 2) - assert.Equal(t, "v2", original["k2"]) - assert.Equal(t, "", original["k3"]) -} diff --git a/modules/generator/remotewrite/config.go b/modules/generator/remotewrite/config.go deleted file mode 100644 index 599b3b3d08e..00000000000 --- a/modules/generator/remotewrite/config.go +++ /dev/null @@ -1,17 +0,0 @@ -package remotewrite - -import ( - "flag" - - "github.com/prometheus/prometheus/config" -) - -type Config struct { - // Enable remote-write requests. If disabled all generated metrics will be discarded. - Enabled bool `yaml:"enabled"` - - Client config.RemoteWriteConfig `yaml:"client"` -} - -func (cfg *Config) RegisterFlagsAndApplyDefaults(prefix string, f *flag.FlagSet) { -} diff --git a/modules/generator/remotewrite/metrics.go b/modules/generator/remotewrite/metrics.go deleted file mode 100644 index 880ee3161bd..00000000000 --- a/modules/generator/remotewrite/metrics.go +++ /dev/null @@ -1,40 +0,0 @@ -package remotewrite - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" -) - -type Metrics struct { - samplesSent *prometheus.CounterVec - exemplarsSent *prometheus.CounterVec - remoteWriteErrors *prometheus.CounterVec - remoteWriteTotal *prometheus.CounterVec -} - -// NewMetrics creates a Metrics and registers all counters with the given prometheus.Registerer. To -// avoid registering metrics twice, this method should only be called once per Registerer. -func NewMetrics(reg prometheus.Registerer) *Metrics { - return &Metrics{ - samplesSent: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "tempo", - Name: "metrics_generator_samples_sent_total", - Help: "Number of samples sent", - }, []string{"tenant"}), - exemplarsSent: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "tempo", - Name: "metrics_generator_exemplars_sent_total", - Help: "Number of exemplars sent", - }, []string{"tenant"}), - remoteWriteErrors: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "tempo", - Name: "metrics_generator_remote_write_errors", - Help: "Number of remote-write requests that failed due to error.", - }, []string{"tenant"}), - remoteWriteTotal: promauto.With(reg).NewCounterVec(prometheus.CounterOpts{ - Namespace: "tempo", - Name: "metrics_generator_remote_write_total", - Help: "Number of remote-write requests.", - }, []string{"tenant"}), - } -} diff --git a/modules/generator/remotewrite/noop.go b/modules/generator/remotewrite/noop.go deleted file mode 100644 index ba8629f7fd7..00000000000 --- a/modules/generator/remotewrite/noop.go +++ /dev/null @@ -1,35 +0,0 @@ -package remotewrite - -import ( - "context" - - "github.com/prometheus/prometheus/model/exemplar" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" -) - -// NoopAppender implements storage.Appendable and storage.Appender -type NoopAppender struct{} - -var _ storage.Appendable = (*NoopAppender)(nil) -var _ storage.Appender = (*NoopAppender)(nil) - -func (a *NoopAppender) Appender(_ context.Context) storage.Appender { - return a -} - -func (a *NoopAppender) Append(_ storage.SeriesRef, _ labels.Labels, _ int64, _ float64) (storage.SeriesRef, error) { - return 0, nil -} - -func (a *NoopAppender) AppendExemplar(_ storage.SeriesRef, _ labels.Labels, _ exemplar.Exemplar) (storage.SeriesRef, error) { - return 0, nil -} - -func (a *NoopAppender) Commit() error { - return nil -} - -func (a *NoopAppender) Rollback() error { - return nil -} diff --git a/modules/generator/storage/config.go b/modules/generator/storage/config.go new file mode 100644 index 00000000000..ab2bac7600c --- /dev/null +++ b/modules/generator/storage/config.go @@ -0,0 +1,66 @@ +package storage + +import ( + "flag" + "time" + + prometheus_config "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/tsdb/agent" +) + +type Config struct { + // Path to store the WAL. Each tenant will be stored in its own subdirectory. + Path string `yaml:"path"` + + Wal agentOptions `yaml:"wal"` + + // How long to wait when flushing sample on shutdown + RemoteWriteFlushDeadline time.Duration `yaml:"remote_write_flush_deadline"` + + // Prometheus remote write config + // https://prometheus.io/docs/prometheus/latest/configuration/configuration/#remote_write + RemoteWrite []*prometheus_config.RemoteWriteConfig `yaml:"remote_write,omitempty"` +} + +func (cfg *Config) RegisterFlagsAndApplyDefaults(prefix string, f *flag.FlagSet) { + cfg.Wal = agentDefaultOptions() + + cfg.RemoteWriteFlushDeadline = time.Minute +} + +// agentOptions is a copy of agent.Options but with yaml struct tags. Refer to agent.Options for +// documentation. +type agentOptions struct { + WALSegmentSize int `yaml:"wal_segment_size"` + WALCompression bool `yaml:"wal_compression"` + StripeSize int `yaml:"stripe_size"` + TruncateFrequency time.Duration `yaml:"truncate_frequency"` + MinWALTime int64 `yaml:"min_wal_time"` + MaxWALTime int64 `yaml:"max_wal_time"` + NoLockfile bool `yaml:"no_lockfile"` +} + +func agentDefaultOptions() agentOptions { + defaultOptions := agent.DefaultOptions() + return agentOptions{ + WALSegmentSize: defaultOptions.WALSegmentSize, + WALCompression: defaultOptions.WALCompression, + StripeSize: defaultOptions.StripeSize, + TruncateFrequency: defaultOptions.TruncateFrequency, + MinWALTime: defaultOptions.MinWALTime, + MaxWALTime: defaultOptions.MaxWALTime, + NoLockfile: defaultOptions.NoLockfile, + } +} + +func (a *agentOptions) toPrometheusAgentOptions() *agent.Options { + return &agent.Options{ + WALSegmentSize: a.WALSegmentSize, + WALCompression: a.WALCompression, + StripeSize: a.StripeSize, + TruncateFrequency: a.TruncateFrequency, + MinWALTime: a.MinWALTime, + MaxWALTime: a.MaxWALTime, + NoLockfile: a.NoLockfile, + } +} diff --git a/modules/generator/storage/config_test.go b/modules/generator/storage/config_test.go new file mode 100644 index 00000000000..45cc2afc67c --- /dev/null +++ b/modules/generator/storage/config_test.go @@ -0,0 +1,52 @@ +package storage + +import ( + "net/url" + "testing" + "time" + + prometheus_common_config "github.com/prometheus/common/config" + prometheus_config "github.com/prometheus/prometheus/config" + "github.com/stretchr/testify/assert" + "gopkg.in/yaml.v2" +) + +func TestConfig(t *testing.T) { + cfgStr := ` +path: /var/wal/tempo +wal: + wal_compression: true +remote_write_flush_deadline: 5m +remote_write: + - url: http://prometheus/api/prom/push + headers: + foo: bar +` + + var cfg Config + cfg.RegisterFlagsAndApplyDefaults("", nil) + + err := yaml.UnmarshalStrict([]byte(cfgStr), &cfg) + assert.NoError(t, err) + + walCfg := agentDefaultOptions() + walCfg.WALCompression = true + + remoteWriteConfig := &prometheus_config.DefaultRemoteWriteConfig + prometheusURL, err := url.Parse("http://prometheus/api/prom/push") + assert.NoError(t, err) + remoteWriteConfig.URL = &prometheus_common_config.URL{URL: prometheusURL} + remoteWriteConfig.Headers = map[string]string{ + "foo": "bar", + } + + expectedCfg := Config{ + Path: "/var/wal/tempo", + Wal: walCfg, + RemoteWriteFlushDeadline: 5 * time.Minute, + RemoteWrite: []*prometheus_config.RemoteWriteConfig{ + remoteWriteConfig, + }, + } + assert.Equal(t, expectedCfg, cfg) +} diff --git a/modules/generator/storage/config_util.go b/modules/generator/storage/config_util.go new file mode 100644 index 00000000000..ff72cf59635 --- /dev/null +++ b/modules/generator/storage/config_util.go @@ -0,0 +1,51 @@ +package storage + +import ( + "strings" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + prometheus_config "github.com/prometheus/prometheus/config" + "github.com/weaveworks/common/user" +) + +// generateTenantRemoteWriteConfigs creates a copy of the remote write configurations with the +// X-Scope-OrgID header present for the given tenant. If the remote write config already contains +// this header it will be overwritten. +func generateTenantRemoteWriteConfigs(originalCfgs []*prometheus_config.RemoteWriteConfig, tenant string, logger log.Logger) []*prometheus_config.RemoteWriteConfig { + var cloneCfgs []*prometheus_config.RemoteWriteConfig + + for _, originalCfg := range originalCfgs { + cloneCfg := &prometheus_config.RemoteWriteConfig{} + *cloneCfg = *originalCfg + + // Copy headers so we can modify them + cloneCfg.Headers = copyMap(cloneCfg.Headers) + + // Ensure that no variation of the X-Scope-OrgId header can be added, which might trick authentication + for k, v := range cloneCfg.Headers { + if strings.EqualFold(user.OrgIDHeaderName, strings.TrimSpace(k)) { + level.Warn(logger).Log("msg", "discarding X-Scope-OrgId header", "key", k, "value", v) + delete(cloneCfg.Headers, k) + } + } + + // inject the X-Scope-OrgId header for multi-tenant metrics backends + cloneCfg.Headers[user.OrgIDHeaderName] = tenant + + cloneCfgs = append(cloneCfgs, cloneCfg) + } + + return cloneCfgs +} + +// copyMap creates a new map containing all values from the given map. +func copyMap(m map[string]string) map[string]string { + newMap := make(map[string]string, len(m)) + + for k, v := range m { + newMap[k] = v + } + + return newMap +} diff --git a/modules/generator/storage/config_util_test.go b/modules/generator/storage/config_util_test.go new file mode 100644 index 00000000000..609df0f1d50 --- /dev/null +++ b/modules/generator/storage/config_util_test.go @@ -0,0 +1,66 @@ +package storage + +import ( + "net/url" + "os" + "testing" + + "github.com/go-kit/log" + prometheus_common_config "github.com/prometheus/common/config" + prometheus_config "github.com/prometheus/prometheus/config" + "github.com/stretchr/testify/assert" +) + +func Test_generateTenantRemoteWriteConfigs(t *testing.T) { + logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stdout)) + + original := []*prometheus_config.RemoteWriteConfig{ + { + URL: &prometheus_common_config.URL{URL: urlMustParse("http://prometheus-1/api/prom/push")}, + Headers: map[string]string{}, + }, + { + URL: &prometheus_common_config.URL{URL: urlMustParse("http://prometheus-2/api/prom/push")}, + Headers: map[string]string{ + "foo": "bar", + "x-scope-orgid": "fake-tenant", + }, + }, + } + + result := generateTenantRemoteWriteConfigs(original, "my-tenant", logger) + + assert.Equal(t, original[0].URL, result[0].URL) + assert.Equal(t, map[string]string{}, original[0].Headers, "Original headers have been modified") + assert.Equal(t, map[string]string{"X-Scope-OrgID": "my-tenant"}, result[0].Headers) + + assert.Equal(t, original[1].URL, result[1].URL) + assert.Equal(t, map[string]string{"foo": "bar", "x-scope-orgid": "fake-tenant"}, original[1].Headers, "Original headers have been modified") + assert.Equal(t, map[string]string{"foo": "bar", "X-Scope-OrgID": "my-tenant"}, result[1].Headers) +} + +func Test_copyMap(t *testing.T) { + original := map[string]string{ + "k1": "v1", + "k2": "v2", + } + + copied := copyMap(original) + + assert.Equal(t, original, copied) + + copied["k2"] = "other value" + copied["k3"] = "v3" + + assert.Len(t, original, 2) + assert.Equal(t, "v2", original["k2"]) + assert.Equal(t, "", original["k3"]) +} + +func urlMustParse(urlStr string) *url.URL { + url, err := url.Parse(urlStr) + if err != nil { + panic(err) + } + return url +} diff --git a/modules/generator/storage/instance.go b/modules/generator/storage/instance.go new file mode 100644 index 00000000000..93cba500654 --- /dev/null +++ b/modules/generator/storage/instance.go @@ -0,0 +1,107 @@ +package storage + +import ( + "context" + "errors" + "os" + "path/filepath" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/common/model" + prometheus_config "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/scrape" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/storage/remote" + "github.com/prometheus/prometheus/tsdb/agent" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" +) + +type Storage interface { + storage.Appendable + + // Close closes the storage and all its underlying resources. + Close() error +} + +type storageImpl struct { + walDir string + wal *agent.DB + remoteStorage *remote.Storage + + logger log.Logger +} + +var _ Storage = (*storageImpl)(nil) + +// New creates a metrics WAL that remote writes its data. +func New(cfg *Config, tenant string, reg prometheus.Registerer, logger log.Logger) (Storage, error) { + logger = log.With(logger, "tenant", tenant) + reg = prometheus.WrapRegistererWith(prometheus.Labels{"tenant": tenant}, reg) + + walDir := filepath.Join(cfg.Path, tenant) + + level.Info(logger).Log("msg", "creating WAL", "dir", walDir) + + // Create WAL directory with necessary permissions + // This creates both // and //wal/. If we don't create the wal + // subdirectory remote storage logs a scary error. + err := os.MkdirAll(filepath.Join(walDir, "wal"), 0o755) + if err != nil { + level.Warn(logger).Log("msg", "could not create directory for metrics WAL", "err", err) + } + + // Set up remote storage writer + startTimeCallback := func() (int64, error) { + return int64(model.Latest), nil + } + remoteStorage := remote.NewStorage(log.With(logger, "component", "remote"), reg, startTimeCallback, walDir, cfg.RemoteWriteFlushDeadline, &noopScrapeManager{}) + + remoteStorageConfig := &prometheus_config.Config{ + RemoteWriteConfigs: generateTenantRemoteWriteConfigs(cfg.RemoteWrite, tenant, logger), + } + + err = remoteStorage.ApplyConfig(remoteStorageConfig) + if err != nil { + return nil, err + } + + // Set up WAL + wal, err := agent.Open(log.With(logger, "component", "wal"), reg, remoteStorage, walDir, cfg.Wal.toPrometheusAgentOptions()) + if err != nil { + return nil, err + } + + return &storageImpl{ + walDir: walDir, + wal: wal, + remoteStorage: remoteStorage, + + logger: logger, + }, nil +} + +func (s *storageImpl) Appender(ctx context.Context) storage.Appender { + return s.wal.Appender(ctx) +} + +func (s *storageImpl) Close() error { + level.Info(s.logger).Log("msg", "closing WAL", "dir", s.walDir) + + return tsdb_errors.NewMulti( + s.wal.Close(), + s.remoteStorage.Close(), + func() error { + // remove the WAL at shutdown since remote write starts at the end of the WAL anyways + // https://github.com/prometheus/prometheus/issues/8809 + return os.RemoveAll(s.walDir) + }(), + ).Err() +} + +type noopScrapeManager struct{} + +func (noop *noopScrapeManager) Get() (*scrape.Manager, error) { + return nil, errors.New("scrape manager not implemented") +} diff --git a/modules/generator/storage/instance_test.go b/modules/generator/storage/instance_test.go new file mode 100644 index 00000000000..dca39ecf124 --- /dev/null +++ b/modules/generator/storage/instance_test.go @@ -0,0 +1,265 @@ +package storage + +import ( + "context" + "fmt" + "net/http" + "net/http/httptest" + "os" + "strconv" + "sync" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + prometheus_common_config "github.com/prometheus/common/config" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/prompb" + "github.com/prometheus/prometheus/storage/remote" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/weaveworks/common/user" + "go.uber.org/atomic" +) + +// Verify basic functionality like sending metrics and exemplars, buffering and retrying failed +// requests. +func TestInstance(t *testing.T) { + var err error + logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stdout)) + + mockServer := newMockPrometheusRemoteWriterServer(logger) + defer mockServer.close() + + var cfg Config + cfg.RegisterFlagsAndApplyDefaults("", nil) + cfg.Path = t.TempDir() + cfg.RemoteWrite = mockServer.remoteWriteConfig() + + instance, err := New(&cfg, "test-tenant", prometheus.DefaultRegisterer, logger) + require.NoError(t, err) + + // Refuse requests - the WAL should buffer data until requests succeed + mockServer.refuseRequests.Store(true) + + sendCtx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Append some data every second + go poll(sendCtx, time.Second, func() { + appender := instance.Appender(context.Background()) + + lbls := labels.FromMap(map[string]string{"__name__": "my-metrics"}) + ref, err := appender.Append(0, lbls, time.Now().UnixMilli(), 1.0) + assert.NoError(t, err) + + _, err = appender.AppendExemplar(ref, lbls, exemplar.Exemplar{ + Labels: labels.FromMap(map[string]string{"traceID": "123"}), + Value: 1.2, + }) + assert.NoError(t, err) + + err = appender.Commit() + assert.NoError(t, err) + }) + + // Wait until remote.Storage has tried at least once to send data + err = waitUntil(10*time.Second, func() bool { + mockServer.mtx.Lock() + defer mockServer.mtx.Unlock() + + return mockServer.refusedRequests > 0 + }) + require.NoError(t, err, "timed out while waiting for refused requests") + + // Allow requests + mockServer.refuseRequests.Store(false) + + // Shutdown the instance - even though previous requests failed, remote.Storage should flush pending data + err = instance.Close() + assert.NoError(t, err) + + // WAL should be empty again + entries, err := os.ReadDir(cfg.Path) + assert.NoError(t, err) + assert.Len(t, entries, 0) + + // Verify we received metrics + assert.Len(t, mockServer.timeSeries, 1) + assert.Contains(t, mockServer.timeSeries, "test-tenant") + // We should have received at least 2 time series: one for the sample and one for the examplar + assert.GreaterOrEqual(t, len(mockServer.timeSeries["test-tenant"]), 2) +} + +// Verify multiple instances function next to each other, don't trample over each other and are isolated. +func TestInstance_multiTenancy(t *testing.T) { + var err error + logger := log.NewLogfmtLogger(log.NewSyncWriter(os.Stdout)) + + mockServer := newMockPrometheusRemoteWriterServer(logger) + defer mockServer.close() + + var cfg Config + cfg.RegisterFlagsAndApplyDefaults("", nil) + cfg.Path = t.TempDir() + cfg.RemoteWrite = mockServer.remoteWriteConfig() + + var instances []Storage + + for i := 0; i < 3; i++ { + instance, err := New(&cfg, strconv.Itoa(i), prometheus.DefaultRegisterer, logger) + assert.NoError(t, err) + instances = append(instances, instance) + } + + sendCtx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // Append some data every second + go poll(sendCtx, time.Second, func() { + for i, instance := range instances { + appender := instance.Appender(context.Background()) + + lbls := labels.FromMap(map[string]string{"__name__": "my-metric"}) + _, err := appender.Append(0, lbls, time.Now().UnixMilli(), float64(i)) + assert.NoError(t, err) + + err = appender.Commit() + assert.NoError(t, err) + } + }) + + // Wait until every tenant received at least one request + err = waitUntil(10*time.Second, func() bool { + mockServer.mtx.Lock() + defer mockServer.mtx.Unlock() + + for i := range instances { + if mockServer.acceptedRequests[strconv.Itoa(i)] == 0 { + return false + } + } + return true + }) + require.NoError(t, err, "timed out while waiting for accepted requests") + + for _, instance := range instances { + // Shutdown the instance - remote write should flush pending data + err = instance.Close() + assert.NoError(t, err) + } + + // WAL should be empty again + entries, err := os.ReadDir(cfg.Path) + assert.NoError(t, err) + require.Len(t, entries, 0) + + for i := range instances { + lenOk := assert.GreaterOrEqual(t, len(mockServer.timeSeries[strconv.Itoa(i)]), 1, "instance %d did not receive the expected amount of time series", i) + if lenOk { + sample := mockServer.timeSeries[strconv.Itoa(i)][0] + assert.Equal(t, float64(i), sample.GetSamples()[0].GetValue()) + } + } +} + +type mockPrometheusRemoteWriteServer struct { + mtx sync.Mutex + + server *httptest.Server + refuseRequests *atomic.Bool + timeSeries map[string][]prompb.TimeSeries + + // metrics + refusedRequests int + acceptedRequests map[string]int + + logger log.Logger +} + +func newMockPrometheusRemoteWriterServer(logger log.Logger) *mockPrometheusRemoteWriteServer { + logger = log.With(logger, "component", "mockserver") + + m := &mockPrometheusRemoteWriteServer{ + refuseRequests: atomic.NewBool(false), + timeSeries: make(map[string][]prompb.TimeSeries), + acceptedRequests: map[string]int{}, + logger: logger, + } + m.server = httptest.NewServer(m) + return m +} + +func (m *mockPrometheusRemoteWriteServer) remoteWriteConfig() []*config.RemoteWriteConfig { + rwCfg := &config.DefaultRemoteWriteConfig + rwCfg.URL = &prometheus_common_config.URL{URL: urlMustParse(fmt.Sprintf("%s/receive", m.server.URL))} + rwCfg.SendExemplars = true + // Aggressive queue settings to speed up tests + rwCfg.QueueConfig.BatchSendDeadline = model.Duration(10 * time.Millisecond) + return []*config.RemoteWriteConfig{rwCfg} +} + +func (m *mockPrometheusRemoteWriteServer) ServeHTTP(res http.ResponseWriter, req *http.Request) { + m.mtx.Lock() + defer m.mtx.Unlock() + + if m.refuseRequests.Load() { + m.logger.Log("msg", "refusing request") + m.refusedRequests++ + http.Error(res, "request refused", http.StatusServiceUnavailable) + return + } + + tenant := req.Header.Get(user.OrgIDHeaderName) + m.logger.Log("msg", "received request", "tenant", tenant) + m.acceptedRequests[tenant]++ + + writeRequest, err := remote.DecodeWriteRequest(req.Body) + if err != nil { + http.Error(res, err.Error(), http.StatusInternalServerError) + return + } + + m.timeSeries[tenant] = append(m.timeSeries[tenant], writeRequest.GetTimeseries()...) +} + +func (m *mockPrometheusRemoteWriteServer) close() { + m.mtx.Lock() + defer m.mtx.Unlock() + + m.server.Close() +} + +// poll executes f every interval until ctx is done or cancelled. +func poll(ctx context.Context, interval time.Duration, f func()) { + ticker := time.NewTicker(interval) + + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + f() + } + } +} + +// waitUntil executes f until it returns true or timeout is reached. +func waitUntil(timeout time.Duration, f func() bool) error { + start := time.Now() + + for { + if f() { + return nil + } + if time.Since(start) > timeout { + return fmt.Errorf("timed out while waiting for condition") + } + + time.Sleep(50 * time.Millisecond) + } +} diff --git a/vendor/github.com/klauspost/compress/snappy/.gitignore b/vendor/github.com/klauspost/compress/snappy/.gitignore deleted file mode 100644 index 042091d9b3b..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/.gitignore +++ /dev/null @@ -1,16 +0,0 @@ -cmd/snappytool/snappytool -testdata/bench - -# These explicitly listed benchmark data files are for an obsolete version of -# snappy_test.go. -testdata/alice29.txt -testdata/asyoulik.txt -testdata/fireworks.jpeg -testdata/geo.protodata -testdata/html -testdata/html_x_4 -testdata/kppkn.gtb -testdata/lcet10.txt -testdata/paper-100k.pdf -testdata/plrabn12.txt -testdata/urls.10K diff --git a/vendor/github.com/klauspost/compress/snappy/AUTHORS b/vendor/github.com/klauspost/compress/snappy/AUTHORS deleted file mode 100644 index 52ccb5a934d..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/AUTHORS +++ /dev/null @@ -1,18 +0,0 @@ -# This is the official list of Snappy-Go authors for copyright purposes. -# This file is distinct from the CONTRIBUTORS files. -# See the latter for an explanation. - -# Names should be added to this file as -# Name or Organization -# The email address is not required for organizations. - -# Please keep the list sorted. - -Amazon.com, Inc -Damian Gryski -Eric Buth -Google Inc. -Jan Mercl <0xjnml@gmail.com> -Klaus Post -Rodolfo Carvalho -Sebastien Binet diff --git a/vendor/github.com/klauspost/compress/snappy/CONTRIBUTORS b/vendor/github.com/klauspost/compress/snappy/CONTRIBUTORS deleted file mode 100644 index ea6524ddd02..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/CONTRIBUTORS +++ /dev/null @@ -1,41 +0,0 @@ -# This is the official list of people who can contribute -# (and typically have contributed) code to the Snappy-Go repository. -# The AUTHORS file lists the copyright holders; this file -# lists people. For example, Google employees are listed here -# but not in AUTHORS, because Google holds the copyright. -# -# The submission process automatically checks to make sure -# that people submitting code are listed in this file (by email address). -# -# Names should be added to this file only after verifying that -# the individual or the individual's organization has agreed to -# the appropriate Contributor License Agreement, found here: -# -# http://code.google.com/legal/individual-cla-v1.0.html -# http://code.google.com/legal/corporate-cla-v1.0.html -# -# The agreement for individuals can be filled out on the web. -# -# When adding J Random Contributor's name to this file, -# either J's name or J's organization's name should be -# added to the AUTHORS file, depending on whether the -# individual or corporate CLA was used. - -# Names should be added to this file like so: -# Name - -# Please keep the list sorted. - -Alex Legg -Damian Gryski -Eric Buth -Jan Mercl <0xjnml@gmail.com> -Jonathan Swinney -Kai Backman -Klaus Post -Marc-Antoine Ruel -Nigel Tao -Rob Pike -Rodolfo Carvalho -Russ Cox -Sebastien Binet diff --git a/vendor/github.com/klauspost/compress/snappy/LICENSE b/vendor/github.com/klauspost/compress/snappy/LICENSE deleted file mode 100644 index 6050c10f4c8..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/LICENSE +++ /dev/null @@ -1,27 +0,0 @@ -Copyright (c) 2011 The Snappy-Go Authors. 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 Google Inc. 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 -OWNER 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. diff --git a/vendor/github.com/klauspost/compress/snappy/README.md b/vendor/github.com/klauspost/compress/snappy/README.md deleted file mode 100644 index 7abf6dfc92c..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/README.md +++ /dev/null @@ -1,17 +0,0 @@ -# snappy - -The Snappy compression format in the Go programming language. - -This is a plug-in replacement for `github.com/golang/snappy`. - -It provides full replacement of the Snappy package. - -See [Snappy Compatibility](https://github.com/klauspost/compress/tree/master/s2#snappy-compatibility) in the S2 documentation. - -"Better" compression mode is used. For buffered streams concurrent compression is used. - -For more options use the [s2 package](https://pkg.go.dev/github.com/klauspost/compress/s2). - -# usage - -Replace imports `github.com/golang/snappy` with `github.com/klauspost/compress/snappy`. diff --git a/vendor/github.com/klauspost/compress/snappy/decode.go b/vendor/github.com/klauspost/compress/snappy/decode.go deleted file mode 100644 index 89f1fa23444..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/decode.go +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright 2011 The Snappy-Go Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package snappy - -import ( - "io" - - "github.com/klauspost/compress/s2" -) - -var ( - // ErrCorrupt reports that the input is invalid. - ErrCorrupt = s2.ErrCorrupt - // ErrTooLarge reports that the uncompressed length is too large. - ErrTooLarge = s2.ErrTooLarge - // ErrUnsupported reports that the input isn't supported. - ErrUnsupported = s2.ErrUnsupported -) - -const ( - // maxBlockSize is the maximum size of the input to encodeBlock. It is not - // part of the wire format per se, but some parts of the encoder assume - // that an offset fits into a uint16. - // - // Also, for the framing format (Writer type instead of Encode function), - // https://github.com/google/snappy/blob/master/framing_format.txt says - // that "the uncompressed data in a chunk must be no longer than 65536 - // bytes". - maxBlockSize = 65536 -) - -// DecodedLen returns the length of the decoded block. -func DecodedLen(src []byte) (int, error) { - return s2.DecodedLen(src) -} - -// Decode returns the decoded form of src. The returned slice may be a sub- -// slice of dst if dst was large enough to hold the entire decoded block. -// Otherwise, a newly allocated slice will be returned. -// -// The dst and src must not overlap. It is valid to pass a nil dst. -// -// Decode handles the Snappy block format, not the Snappy stream format. -func Decode(dst, src []byte) ([]byte, error) { - return s2.Decode(dst, src) -} - -// NewReader returns a new Reader that decompresses from r, using the framing -// format described at -// https://github.com/google/snappy/blob/master/framing_format.txt -func NewReader(r io.Reader) *Reader { - return s2.NewReader(r, s2.ReaderMaxBlockSize(maxBlockSize)) -} - -// Reader is an io.Reader that can read Snappy-compressed bytes. -// -// Reader handles the Snappy stream format, not the Snappy block format. -type Reader = s2.Reader diff --git a/vendor/github.com/klauspost/compress/snappy/encode.go b/vendor/github.com/klauspost/compress/snappy/encode.go deleted file mode 100644 index e8bd72c1864..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/encode.go +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright 2011 The Snappy-Go Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -package snappy - -import ( - "io" - - "github.com/klauspost/compress/s2" -) - -// Encode returns the encoded form of src. The returned slice may be a sub- -// slice of dst if dst was large enough to hold the entire encoded block. -// Otherwise, a newly allocated slice will be returned. -// -// The dst and src must not overlap. It is valid to pass a nil dst. -// -// Encode handles the Snappy block format, not the Snappy stream format. -func Encode(dst, src []byte) []byte { - return s2.EncodeSnappyBetter(dst, src) -} - -// MaxEncodedLen returns the maximum length of a snappy block, given its -// uncompressed length. -// -// It will return a negative value if srcLen is too large to encode. -func MaxEncodedLen(srcLen int) int { - return s2.MaxEncodedLen(srcLen) -} - -// NewWriter returns a new Writer that compresses to w. -// -// The Writer returned does not buffer writes. There is no need to Flush or -// Close such a Writer. -// -// Deprecated: the Writer returned is not suitable for many small writes, only -// for few large writes. Use NewBufferedWriter instead, which is efficient -// regardless of the frequency and shape of the writes, and remember to Close -// that Writer when done. -func NewWriter(w io.Writer) *Writer { - return s2.NewWriter(w, s2.WriterSnappyCompat(), s2.WriterBetterCompression(), s2.WriterFlushOnWrite(), s2.WriterConcurrency(1)) -} - -// NewBufferedWriter returns a new Writer that compresses to w, using the -// framing format described at -// https://github.com/google/snappy/blob/master/framing_format.txt -// -// The Writer returned buffers writes. Users must call Close to guarantee all -// data has been forwarded to the underlying io.Writer. They may also call -// Flush zero or more times before calling Close. -func NewBufferedWriter(w io.Writer) *Writer { - return s2.NewWriter(w, s2.WriterSnappyCompat(), s2.WriterBetterCompression()) -} - -// Writer is an io.Writer that can write Snappy-compressed bytes. -// -// Writer handles the Snappy stream format, not the Snappy block format. -type Writer = s2.Writer diff --git a/vendor/github.com/klauspost/compress/snappy/snappy.go b/vendor/github.com/klauspost/compress/snappy/snappy.go deleted file mode 100644 index 398cdc95a01..00000000000 --- a/vendor/github.com/klauspost/compress/snappy/snappy.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2011 The Snappy-Go Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSE file. - -// Package snappy implements the Snappy compression format. It aims for very -// high speeds and reasonable compression. -// -// There are actually two Snappy formats: block and stream. They are related, -// but different: trying to decompress block-compressed data as a Snappy stream -// will fail, and vice versa. The block format is the Decode and Encode -// functions and the stream format is the Reader and Writer types. -// -// The block format, the more common case, is used when the complete size (the -// number of bytes) of the original data is known upfront, at the time -// compression starts. The stream format, also known as the framing format, is -// for when that isn't always true. -// -// The canonical, C++ implementation is at https://github.com/google/snappy and -// it only implements the block format. -package snappy - -/* -Each encoded block begins with the varint-encoded length of the decoded data, -followed by a sequence of chunks. Chunks begin and end on byte boundaries. The -first byte of each chunk is broken into its 2 least and 6 most significant bits -called l and m: l ranges in [0, 4) and m ranges in [0, 64). l is the chunk tag. -Zero means a literal tag. All other values mean a copy tag. - -For literal tags: - - If m < 60, the next 1 + m bytes are literal bytes. - - Otherwise, let n be the little-endian unsigned integer denoted by the next - m - 59 bytes. The next 1 + n bytes after that are literal bytes. - -For copy tags, length bytes are copied from offset bytes ago, in the style of -Lempel-Ziv compression algorithms. In particular: - - For l == 1, the offset ranges in [0, 1<<11) and the length in [4, 12). - The length is 4 + the low 3 bits of m. The high 3 bits of m form bits 8-10 - of the offset. The next byte is bits 0-7 of the offset. - - For l == 2, the offset ranges in [0, 1<<16) and the length in [1, 65). - The length is 1 + m. The offset is the little-endian unsigned integer - denoted by the next 2 bytes. - - For l == 3, this tag is a legacy format that is no longer issued by most - encoders. Nonetheless, the offset ranges in [0, 1<<32) and the length in - [1, 65). The length is 1 + m. The offset is the little-endian unsigned - integer denoted by the next 4 bytes. -*/ diff --git a/vendor/github.com/oklog/ulid/.gitignore b/vendor/github.com/oklog/ulid/.gitignore new file mode 100644 index 00000000000..c92c4d56084 --- /dev/null +++ b/vendor/github.com/oklog/ulid/.gitignore @@ -0,0 +1,29 @@ +#### joe made this: http://goel.io/joe + +#####=== Go ===##### + +# Compiled Object files, Static and Dynamic libs (Shared Objects) +*.o +*.a +*.so + +# Folders +_obj +_test + +# Architecture specific extensions/prefixes +*.[568vq] +[568vq].out + +*.cgo1.go +*.cgo2.c +_cgo_defun.c +_cgo_gotypes.go +_cgo_export.* + +_testmain.go + +*.exe +*.test +*.prof + diff --git a/vendor/github.com/oklog/ulid/.travis.yml b/vendor/github.com/oklog/ulid/.travis.yml new file mode 100644 index 00000000000..43eb762fa34 --- /dev/null +++ b/vendor/github.com/oklog/ulid/.travis.yml @@ -0,0 +1,16 @@ +language: go +sudo: false +go: + - 1.10.x +install: + - go get -v github.com/golang/lint/golint + - go get golang.org/x/tools/cmd/cover + - go get github.com/mattn/goveralls + - go get -d -t -v ./... + - go build -v ./... +script: + - go vet ./... + - $HOME/gopath/bin/golint . + - go test -v -race ./... + - go test -v -covermode=count -coverprofile=cov.out + - $HOME/gopath/bin/goveralls -coverprofile=cov.out -service=travis-ci -repotoken "$COVERALLS_TOKEN" || true diff --git a/vendor/github.com/oklog/ulid/AUTHORS.md b/vendor/github.com/oklog/ulid/AUTHORS.md new file mode 100644 index 00000000000..95581c78b06 --- /dev/null +++ b/vendor/github.com/oklog/ulid/AUTHORS.md @@ -0,0 +1,2 @@ +- Peter Bourgon (@peterbourgon) +- Tomás Senart (@tsenart) diff --git a/vendor/github.com/oklog/ulid/CHANGELOG.md b/vendor/github.com/oklog/ulid/CHANGELOG.md new file mode 100644 index 00000000000..8da38c6b00d --- /dev/null +++ b/vendor/github.com/oklog/ulid/CHANGELOG.md @@ -0,0 +1,33 @@ +## 1.3.1 / 2018-10-02 + +* Use underlying entropy source for random increments in Monotonic (#32) + +## 1.3.0 / 2018-09-29 + +* Monotonic entropy support (#31) + +## 1.2.0 / 2018-09-09 + +* Add a function to convert Unix time in milliseconds back to time.Time (#30) + +## 1.1.0 / 2018-08-15 + +* Ensure random part is always read from the entropy reader in full (#28) + +## 1.0.0 / 2018-07-29 + +* Add ParseStrict and MustParseStrict functions (#26) +* Enforce overflow checking when parsing (#20) + +## 0.3.0 / 2017-01-03 + +* Implement ULID.Compare method + +## 0.2.0 / 2016-12-13 + +* Remove year 2262 Timestamp bug. (#1) +* Gracefully handle invalid encodings when parsing. + +## 0.1.0 / 2016-12-06 + +* First ULID release diff --git a/vendor/github.com/oklog/ulid/CONTRIBUTING.md b/vendor/github.com/oklog/ulid/CONTRIBUTING.md new file mode 100644 index 00000000000..68f03f26eba --- /dev/null +++ b/vendor/github.com/oklog/ulid/CONTRIBUTING.md @@ -0,0 +1,17 @@ +# Contributing + +We use GitHub to manage reviews of pull requests. + +* If you have a trivial fix or improvement, go ahead and create a pull + request, addressing (with `@...`) one or more of the maintainers + (see [AUTHORS.md](AUTHORS.md)) in the description of the pull request. + +* If you plan to do something more involved, first propose your ideas + in a Github issue. This will avoid unnecessary work and surely give + you and us a good deal of inspiration. + +* Relevant coding style guidelines are the [Go Code Review + Comments](https://code.google.com/p/go-wiki/wiki/CodeReviewComments) + and the _Formatting and style_ section of Peter Bourgon's [Go: Best + Practices for Production + Environments](http://peter.bourgon.org/go-in-production/#formatting-and-style). diff --git a/vendor/github.com/oklog/ulid/Gopkg.lock b/vendor/github.com/oklog/ulid/Gopkg.lock new file mode 100644 index 00000000000..349b449a6ea --- /dev/null +++ b/vendor/github.com/oklog/ulid/Gopkg.lock @@ -0,0 +1,15 @@ +# This file is autogenerated, do not edit; changes may be undone by the next 'dep ensure'. + + +[[projects]] + branch = "master" + name = "github.com/pborman/getopt" + packages = ["v2"] + revision = "7148bc3a4c3008adfcab60cbebfd0576018f330b" + +[solve-meta] + analyzer-name = "dep" + analyzer-version = 1 + inputs-digest = "6779b05abd5cd429c5393641d2453005a3cb74a400d161b2b5c5d0ca2e10e116" + solver-name = "gps-cdcl" + solver-version = 1 diff --git a/vendor/github.com/oklog/ulid/Gopkg.toml b/vendor/github.com/oklog/ulid/Gopkg.toml new file mode 100644 index 00000000000..624a7a019c7 --- /dev/null +++ b/vendor/github.com/oklog/ulid/Gopkg.toml @@ -0,0 +1,26 @@ + +# Gopkg.toml example +# +# Refer to https://github.com/golang/dep/blob/master/docs/Gopkg.toml.md +# for detailed Gopkg.toml documentation. +# +# required = ["github.com/user/thing/cmd/thing"] +# ignored = ["github.com/user/project/pkgX", "bitbucket.org/user/project/pkgA/pkgY"] +# +# [[constraint]] +# name = "github.com/user/project" +# version = "1.0.0" +# +# [[constraint]] +# name = "github.com/user/project2" +# branch = "dev" +# source = "github.com/myfork/project2" +# +# [[override]] +# name = "github.com/x/y" +# version = "2.4.0" + + +[[constraint]] + branch = "master" + name = "github.com/pborman/getopt" diff --git a/vendor/github.com/oklog/ulid/LICENSE b/vendor/github.com/oklog/ulid/LICENSE new file mode 100644 index 00000000000..261eeb9e9f8 --- /dev/null +++ b/vendor/github.com/oklog/ulid/LICENSE @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. diff --git a/vendor/github.com/oklog/ulid/README.md b/vendor/github.com/oklog/ulid/README.md new file mode 100644 index 00000000000..0a3d2f82b25 --- /dev/null +++ b/vendor/github.com/oklog/ulid/README.md @@ -0,0 +1,150 @@ +# Universally Unique Lexicographically Sortable Identifier + +![Project status](https://img.shields.io/badge/version-1.3.0-yellow.svg) +[![Build Status](https://secure.travis-ci.org/oklog/ulid.png)](http://travis-ci.org/oklog/ulid) +[![Go Report Card](https://goreportcard.com/badge/oklog/ulid?cache=0)](https://goreportcard.com/report/oklog/ulid) +[![Coverage Status](https://coveralls.io/repos/github/oklog/ulid/badge.svg?branch=master&cache=0)](https://coveralls.io/github/oklog/ulid?branch=master) +[![GoDoc](https://godoc.org/github.com/oklog/ulid?status.svg)](https://godoc.org/github.com/oklog/ulid) +[![Apache 2 licensed](https://img.shields.io/badge/license-Apache2-blue.svg)](https://raw.githubusercontent.com/oklog/ulid/master/LICENSE) + +A Go port of [alizain/ulid](https://github.com/alizain/ulid) with binary format implemented. + +## Background + +A GUID/UUID can be suboptimal for many use-cases because: + +- It isn't the most character efficient way of encoding 128 bits +- UUID v1/v2 is impractical in many environments, as it requires access to a unique, stable MAC address +- UUID v3/v5 requires a unique seed and produces randomly distributed IDs, which can cause fragmentation in many data structures +- UUID v4 provides no other information than randomness which can cause fragmentation in many data structures + +A ULID however: + +- Is compatible with UUID/GUID's +- 1.21e+24 unique ULIDs per millisecond (1,208,925,819,614,629,174,706,176 to be exact) +- Lexicographically sortable +- Canonically encoded as a 26 character string, as opposed to the 36 character UUID +- Uses Crockford's base32 for better efficiency and readability (5 bits per character) +- Case insensitive +- No special characters (URL safe) +- Monotonic sort order (correctly detects and handles the same millisecond) + +## Install + +```shell +go get github.com/oklog/ulid +``` + +## Usage + +An ULID is constructed with a `time.Time` and an `io.Reader` entropy source. +This design allows for greater flexibility in choosing your trade-offs. + +Please note that `rand.Rand` from the `math` package is *not* safe for concurrent use. +Instantiate one per long living go-routine or use a `sync.Pool` if you want to avoid the potential contention of a locked `rand.Source` as its been frequently observed in the package level functions. + + +```go +func ExampleULID() { + t := time.Unix(1000000, 0) + entropy := ulid.Monotonic(rand.New(rand.NewSource(t.UnixNano())), 0) + fmt.Println(ulid.MustNew(ulid.Timestamp(t), entropy)) + // Output: 0000XSNJG0MQJHBF4QX1EFD6Y3 +} + +``` + +## Specification + +Below is the current specification of ULID as implemented in this repository. + +### Components + +**Timestamp** +- 48 bits +- UNIX-time in milliseconds +- Won't run out of space till the year 10895 AD + +**Entropy** +- 80 bits +- User defined entropy source. +- Monotonicity within the same millisecond with [`ulid.Monotonic`](https://godoc.org/github.com/oklog/ulid#Monotonic) + +### Encoding + +[Crockford's Base32](http://www.crockford.com/wrmg/base32.html) is used as shown. +This alphabet excludes the letters I, L, O, and U to avoid confusion and abuse. + +``` +0123456789ABCDEFGHJKMNPQRSTVWXYZ +``` + +### Binary Layout and Byte Order + +The components are encoded as 16 octets. Each component is encoded with the Most Significant Byte first (network byte order). + +``` +0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 ++-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +| 32_bit_uint_time_high | ++-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +| 16_bit_uint_time_low | 16_bit_uint_random | ++-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +| 32_bit_uint_random | ++-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +| 32_bit_uint_random | ++-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +``` + +### String Representation + +``` + 01AN4Z07BY 79KA1307SR9X4MV3 +|----------| |----------------| + Timestamp Entropy + 10 chars 16 chars + 48bits 80bits + base32 base32 +``` + +## Test + +```shell +go test ./... +``` + +## Benchmarks + +On a Intel Core i7 Ivy Bridge 2.7 GHz, MacOS 10.12.1 and Go 1.8.0beta1 + +``` +BenchmarkNew/WithCryptoEntropy-8 2000000 771 ns/op 20.73 MB/s 16 B/op 1 allocs/op +BenchmarkNew/WithEntropy-8 20000000 65.8 ns/op 243.01 MB/s 16 B/op 1 allocs/op +BenchmarkNew/WithoutEntropy-8 50000000 30.0 ns/op 534.06 MB/s 16 B/op 1 allocs/op +BenchmarkMustNew/WithCryptoEntropy-8 2000000 781 ns/op 20.48 MB/s 16 B/op 1 allocs/op +BenchmarkMustNew/WithEntropy-8 20000000 70.0 ns/op 228.51 MB/s 16 B/op 1 allocs/op +BenchmarkMustNew/WithoutEntropy-8 50000000 34.6 ns/op 462.98 MB/s 16 B/op 1 allocs/op +BenchmarkParse-8 50000000 30.0 ns/op 866.16 MB/s 0 B/op 0 allocs/op +BenchmarkMustParse-8 50000000 35.2 ns/op 738.94 MB/s 0 B/op 0 allocs/op +BenchmarkString-8 20000000 64.9 ns/op 246.40 MB/s 32 B/op 1 allocs/op +BenchmarkMarshal/Text-8 20000000 55.8 ns/op 286.84 MB/s 32 B/op 1 allocs/op +BenchmarkMarshal/TextTo-8 100000000 22.4 ns/op 714.91 MB/s 0 B/op 0 allocs/op +BenchmarkMarshal/Binary-8 300000000 4.02 ns/op 3981.77 MB/s 0 B/op 0 allocs/op +BenchmarkMarshal/BinaryTo-8 2000000000 1.18 ns/op 13551.75 MB/s 0 B/op 0 allocs/op +BenchmarkUnmarshal/Text-8 100000000 20.5 ns/op 1265.27 MB/s 0 B/op 0 allocs/op +BenchmarkUnmarshal/Binary-8 300000000 4.94 ns/op 3240.01 MB/s 0 B/op 0 allocs/op +BenchmarkNow-8 100000000 15.1 ns/op 528.09 MB/s 0 B/op 0 allocs/op +BenchmarkTimestamp-8 2000000000 0.29 ns/op 27271.59 MB/s 0 B/op 0 allocs/op +BenchmarkTime-8 2000000000 0.58 ns/op 13717.80 MB/s 0 B/op 0 allocs/op +BenchmarkSetTime-8 2000000000 0.89 ns/op 9023.95 MB/s 0 B/op 0 allocs/op +BenchmarkEntropy-8 200000000 7.62 ns/op 1311.66 MB/s 0 B/op 0 allocs/op +BenchmarkSetEntropy-8 2000000000 0.88 ns/op 11376.54 MB/s 0 B/op 0 allocs/op +BenchmarkCompare-8 200000000 7.34 ns/op 4359.23 MB/s 0 B/op 0 allocs/op +``` + +## Prior Art + +- [alizain/ulid](https://github.com/alizain/ulid) +- [RobThree/NUlid](https://github.com/RobThree/NUlid) +- [imdario/go-ulid](https://github.com/imdario/go-ulid) diff --git a/vendor/github.com/oklog/ulid/ulid.go b/vendor/github.com/oklog/ulid/ulid.go new file mode 100644 index 00000000000..c5d0d66fd2a --- /dev/null +++ b/vendor/github.com/oklog/ulid/ulid.go @@ -0,0 +1,614 @@ +// Copyright 2016 The Oklog Authors +// 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 ulid + +import ( + "bufio" + "bytes" + "database/sql/driver" + "encoding/binary" + "errors" + "io" + "math" + "math/bits" + "math/rand" + "time" +) + +/* +An ULID is a 16 byte Universally Unique Lexicographically Sortable Identifier + + The components are encoded as 16 octets. + Each component is encoded with the MSB first (network byte order). + + 0 1 2 3 + 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 + +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + | 32_bit_uint_time_high | + +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + | 16_bit_uint_time_low | 16_bit_uint_random | + +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + | 32_bit_uint_random | + +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ + | 32_bit_uint_random | + +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ +*/ +type ULID [16]byte + +var ( + // ErrDataSize is returned when parsing or unmarshaling ULIDs with the wrong + // data size. + ErrDataSize = errors.New("ulid: bad data size when unmarshaling") + + // ErrInvalidCharacters is returned when parsing or unmarshaling ULIDs with + // invalid Base32 encodings. + ErrInvalidCharacters = errors.New("ulid: bad data characters when unmarshaling") + + // ErrBufferSize is returned when marshalling ULIDs to a buffer of insufficient + // size. + ErrBufferSize = errors.New("ulid: bad buffer size when marshaling") + + // ErrBigTime is returned when constructing an ULID with a time that is larger + // than MaxTime. + ErrBigTime = errors.New("ulid: time too big") + + // ErrOverflow is returned when unmarshaling a ULID whose first character is + // larger than 7, thereby exceeding the valid bit depth of 128. + ErrOverflow = errors.New("ulid: overflow when unmarshaling") + + // ErrMonotonicOverflow is returned by a Monotonic entropy source when + // incrementing the previous ULID's entropy bytes would result in overflow. + ErrMonotonicOverflow = errors.New("ulid: monotonic entropy overflow") + + // ErrScanValue is returned when the value passed to scan cannot be unmarshaled + // into the ULID. + ErrScanValue = errors.New("ulid: source value must be a string or byte slice") +) + +// New returns an ULID with the given Unix milliseconds timestamp and an +// optional entropy source. Use the Timestamp function to convert +// a time.Time to Unix milliseconds. +// +// ErrBigTime is returned when passing a timestamp bigger than MaxTime. +// Reading from the entropy source may also return an error. +func New(ms uint64, entropy io.Reader) (id ULID, err error) { + if err = id.SetTime(ms); err != nil { + return id, err + } + + switch e := entropy.(type) { + case nil: + return id, err + case *monotonic: + err = e.MonotonicRead(ms, id[6:]) + default: + _, err = io.ReadFull(e, id[6:]) + } + + return id, err +} + +// MustNew is a convenience function equivalent to New that panics on failure +// instead of returning an error. +func MustNew(ms uint64, entropy io.Reader) ULID { + id, err := New(ms, entropy) + if err != nil { + panic(err) + } + return id +} + +// Parse parses an encoded ULID, returning an error in case of failure. +// +// ErrDataSize is returned if the len(ulid) is different from an encoded +// ULID's length. Invalid encodings produce undefined ULIDs. For a version that +// returns an error instead, see ParseStrict. +func Parse(ulid string) (id ULID, err error) { + return id, parse([]byte(ulid), false, &id) +} + +// ParseStrict parses an encoded ULID, returning an error in case of failure. +// +// It is like Parse, but additionally validates that the parsed ULID consists +// only of valid base32 characters. It is slightly slower than Parse. +// +// ErrDataSize is returned if the len(ulid) is different from an encoded +// ULID's length. Invalid encodings return ErrInvalidCharacters. +func ParseStrict(ulid string) (id ULID, err error) { + return id, parse([]byte(ulid), true, &id) +} + +func parse(v []byte, strict bool, id *ULID) error { + // Check if a base32 encoded ULID is the right length. + if len(v) != EncodedSize { + return ErrDataSize + } + + // Check if all the characters in a base32 encoded ULID are part of the + // expected base32 character set. + if strict && + (dec[v[0]] == 0xFF || + dec[v[1]] == 0xFF || + dec[v[2]] == 0xFF || + dec[v[3]] == 0xFF || + dec[v[4]] == 0xFF || + dec[v[5]] == 0xFF || + dec[v[6]] == 0xFF || + dec[v[7]] == 0xFF || + dec[v[8]] == 0xFF || + dec[v[9]] == 0xFF || + dec[v[10]] == 0xFF || + dec[v[11]] == 0xFF || + dec[v[12]] == 0xFF || + dec[v[13]] == 0xFF || + dec[v[14]] == 0xFF || + dec[v[15]] == 0xFF || + dec[v[16]] == 0xFF || + dec[v[17]] == 0xFF || + dec[v[18]] == 0xFF || + dec[v[19]] == 0xFF || + dec[v[20]] == 0xFF || + dec[v[21]] == 0xFF || + dec[v[22]] == 0xFF || + dec[v[23]] == 0xFF || + dec[v[24]] == 0xFF || + dec[v[25]] == 0xFF) { + return ErrInvalidCharacters + } + + // Check if the first character in a base32 encoded ULID will overflow. This + // happens because the base32 representation encodes 130 bits, while the + // ULID is only 128 bits. + // + // See https://github.com/oklog/ulid/issues/9 for details. + if v[0] > '7' { + return ErrOverflow + } + + // Use an optimized unrolled loop (from https://github.com/RobThree/NUlid) + // to decode a base32 ULID. + + // 6 bytes timestamp (48 bits) + (*id)[0] = ((dec[v[0]] << 5) | dec[v[1]]) + (*id)[1] = ((dec[v[2]] << 3) | (dec[v[3]] >> 2)) + (*id)[2] = ((dec[v[3]] << 6) | (dec[v[4]] << 1) | (dec[v[5]] >> 4)) + (*id)[3] = ((dec[v[5]] << 4) | (dec[v[6]] >> 1)) + (*id)[4] = ((dec[v[6]] << 7) | (dec[v[7]] << 2) | (dec[v[8]] >> 3)) + (*id)[5] = ((dec[v[8]] << 5) | dec[v[9]]) + + // 10 bytes of entropy (80 bits) + (*id)[6] = ((dec[v[10]] << 3) | (dec[v[11]] >> 2)) + (*id)[7] = ((dec[v[11]] << 6) | (dec[v[12]] << 1) | (dec[v[13]] >> 4)) + (*id)[8] = ((dec[v[13]] << 4) | (dec[v[14]] >> 1)) + (*id)[9] = ((dec[v[14]] << 7) | (dec[v[15]] << 2) | (dec[v[16]] >> 3)) + (*id)[10] = ((dec[v[16]] << 5) | dec[v[17]]) + (*id)[11] = ((dec[v[18]] << 3) | dec[v[19]]>>2) + (*id)[12] = ((dec[v[19]] << 6) | (dec[v[20]] << 1) | (dec[v[21]] >> 4)) + (*id)[13] = ((dec[v[21]] << 4) | (dec[v[22]] >> 1)) + (*id)[14] = ((dec[v[22]] << 7) | (dec[v[23]] << 2) | (dec[v[24]] >> 3)) + (*id)[15] = ((dec[v[24]] << 5) | dec[v[25]]) + + return nil +} + +// MustParse is a convenience function equivalent to Parse that panics on failure +// instead of returning an error. +func MustParse(ulid string) ULID { + id, err := Parse(ulid) + if err != nil { + panic(err) + } + return id +} + +// MustParseStrict is a convenience function equivalent to ParseStrict that +// panics on failure instead of returning an error. +func MustParseStrict(ulid string) ULID { + id, err := ParseStrict(ulid) + if err != nil { + panic(err) + } + return id +} + +// String returns a lexicographically sortable string encoded ULID +// (26 characters, non-standard base 32) e.g. 01AN4Z07BY79KA1307SR9X4MV3 +// Format: tttttttttteeeeeeeeeeeeeeee where t is time and e is entropy +func (id ULID) String() string { + ulid := make([]byte, EncodedSize) + _ = id.MarshalTextTo(ulid) + return string(ulid) +} + +// MarshalBinary implements the encoding.BinaryMarshaler interface by +// returning the ULID as a byte slice. +func (id ULID) MarshalBinary() ([]byte, error) { + ulid := make([]byte, len(id)) + return ulid, id.MarshalBinaryTo(ulid) +} + +// MarshalBinaryTo writes the binary encoding of the ULID to the given buffer. +// ErrBufferSize is returned when the len(dst) != 16. +func (id ULID) MarshalBinaryTo(dst []byte) error { + if len(dst) != len(id) { + return ErrBufferSize + } + + copy(dst, id[:]) + return nil +} + +// UnmarshalBinary implements the encoding.BinaryUnmarshaler interface by +// copying the passed data and converting it to an ULID. ErrDataSize is +// returned if the data length is different from ULID length. +func (id *ULID) UnmarshalBinary(data []byte) error { + if len(data) != len(*id) { + return ErrDataSize + } + + copy((*id)[:], data) + return nil +} + +// Encoding is the base 32 encoding alphabet used in ULID strings. +const Encoding = "0123456789ABCDEFGHJKMNPQRSTVWXYZ" + +// MarshalText implements the encoding.TextMarshaler interface by +// returning the string encoded ULID. +func (id ULID) MarshalText() ([]byte, error) { + ulid := make([]byte, EncodedSize) + return ulid, id.MarshalTextTo(ulid) +} + +// MarshalTextTo writes the ULID as a string to the given buffer. +// ErrBufferSize is returned when the len(dst) != 26. +func (id ULID) MarshalTextTo(dst []byte) error { + // Optimized unrolled loop ahead. + // From https://github.com/RobThree/NUlid + + if len(dst) != EncodedSize { + return ErrBufferSize + } + + // 10 byte timestamp + dst[0] = Encoding[(id[0]&224)>>5] + dst[1] = Encoding[id[0]&31] + dst[2] = Encoding[(id[1]&248)>>3] + dst[3] = Encoding[((id[1]&7)<<2)|((id[2]&192)>>6)] + dst[4] = Encoding[(id[2]&62)>>1] + dst[5] = Encoding[((id[2]&1)<<4)|((id[3]&240)>>4)] + dst[6] = Encoding[((id[3]&15)<<1)|((id[4]&128)>>7)] + dst[7] = Encoding[(id[4]&124)>>2] + dst[8] = Encoding[((id[4]&3)<<3)|((id[5]&224)>>5)] + dst[9] = Encoding[id[5]&31] + + // 16 bytes of entropy + dst[10] = Encoding[(id[6]&248)>>3] + dst[11] = Encoding[((id[6]&7)<<2)|((id[7]&192)>>6)] + dst[12] = Encoding[(id[7]&62)>>1] + dst[13] = Encoding[((id[7]&1)<<4)|((id[8]&240)>>4)] + dst[14] = Encoding[((id[8]&15)<<1)|((id[9]&128)>>7)] + dst[15] = Encoding[(id[9]&124)>>2] + dst[16] = Encoding[((id[9]&3)<<3)|((id[10]&224)>>5)] + dst[17] = Encoding[id[10]&31] + dst[18] = Encoding[(id[11]&248)>>3] + dst[19] = Encoding[((id[11]&7)<<2)|((id[12]&192)>>6)] + dst[20] = Encoding[(id[12]&62)>>1] + dst[21] = Encoding[((id[12]&1)<<4)|((id[13]&240)>>4)] + dst[22] = Encoding[((id[13]&15)<<1)|((id[14]&128)>>7)] + dst[23] = Encoding[(id[14]&124)>>2] + dst[24] = Encoding[((id[14]&3)<<3)|((id[15]&224)>>5)] + dst[25] = Encoding[id[15]&31] + + return nil +} + +// Byte to index table for O(1) lookups when unmarshaling. +// We use 0xFF as sentinel value for invalid indexes. +var dec = [...]byte{ + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x00, 0x01, + 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x0A, 0x0B, 0x0C, 0x0D, 0x0E, + 0x0F, 0x10, 0x11, 0xFF, 0x12, 0x13, 0xFF, 0x14, 0x15, 0xFF, + 0x16, 0x17, 0x18, 0x19, 0x1A, 0xFF, 0x1B, 0x1C, 0x1D, 0x1E, + 0x1F, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0x0A, 0x0B, 0x0C, + 0x0D, 0x0E, 0x0F, 0x10, 0x11, 0xFF, 0x12, 0x13, 0xFF, 0x14, + 0x15, 0xFF, 0x16, 0x17, 0x18, 0x19, 0x1A, 0xFF, 0x1B, 0x1C, + 0x1D, 0x1E, 0x1F, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, + 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, +} + +// EncodedSize is the length of a text encoded ULID. +const EncodedSize = 26 + +// UnmarshalText implements the encoding.TextUnmarshaler interface by +// parsing the data as string encoded ULID. +// +// ErrDataSize is returned if the len(v) is different from an encoded +// ULID's length. Invalid encodings produce undefined ULIDs. +func (id *ULID) UnmarshalText(v []byte) error { + return parse(v, false, id) +} + +// Time returns the Unix time in milliseconds encoded in the ULID. +// Use the top level Time function to convert the returned value to +// a time.Time. +func (id ULID) Time() uint64 { + return uint64(id[5]) | uint64(id[4])<<8 | + uint64(id[3])<<16 | uint64(id[2])<<24 | + uint64(id[1])<<32 | uint64(id[0])<<40 +} + +// maxTime is the maximum Unix time in milliseconds that can be +// represented in an ULID. +var maxTime = ULID{0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF}.Time() + +// MaxTime returns the maximum Unix time in milliseconds that +// can be encoded in an ULID. +func MaxTime() uint64 { return maxTime } + +// Now is a convenience function that returns the current +// UTC time in Unix milliseconds. Equivalent to: +// Timestamp(time.Now().UTC()) +func Now() uint64 { return Timestamp(time.Now().UTC()) } + +// Timestamp converts a time.Time to Unix milliseconds. +// +// Because of the way ULID stores time, times from the year +// 10889 produces undefined results. +func Timestamp(t time.Time) uint64 { + return uint64(t.Unix())*1000 + + uint64(t.Nanosecond()/int(time.Millisecond)) +} + +// Time converts Unix milliseconds in the format +// returned by the Timestamp function to a time.Time. +func Time(ms uint64) time.Time { + s := int64(ms / 1e3) + ns := int64((ms % 1e3) * 1e6) + return time.Unix(s, ns) +} + +// SetTime sets the time component of the ULID to the given Unix time +// in milliseconds. +func (id *ULID) SetTime(ms uint64) error { + if ms > maxTime { + return ErrBigTime + } + + (*id)[0] = byte(ms >> 40) + (*id)[1] = byte(ms >> 32) + (*id)[2] = byte(ms >> 24) + (*id)[3] = byte(ms >> 16) + (*id)[4] = byte(ms >> 8) + (*id)[5] = byte(ms) + + return nil +} + +// Entropy returns the entropy from the ULID. +func (id ULID) Entropy() []byte { + e := make([]byte, 10) + copy(e, id[6:]) + return e +} + +// SetEntropy sets the ULID entropy to the passed byte slice. +// ErrDataSize is returned if len(e) != 10. +func (id *ULID) SetEntropy(e []byte) error { + if len(e) != 10 { + return ErrDataSize + } + + copy((*id)[6:], e) + return nil +} + +// Compare returns an integer comparing id and other lexicographically. +// The result will be 0 if id==other, -1 if id < other, and +1 if id > other. +func (id ULID) Compare(other ULID) int { + return bytes.Compare(id[:], other[:]) +} + +// Scan implements the sql.Scanner interface. It supports scanning +// a string or byte slice. +func (id *ULID) Scan(src interface{}) error { + switch x := src.(type) { + case nil: + return nil + case string: + return id.UnmarshalText([]byte(x)) + case []byte: + return id.UnmarshalBinary(x) + } + + return ErrScanValue +} + +// Value implements the sql/driver.Valuer interface. This returns the value +// represented as a byte slice. If instead a string is desirable, a wrapper +// type can be created that calls String(). +// +// // stringValuer wraps a ULID as a string-based driver.Valuer. +// type stringValuer ULID +// +// func (id stringValuer) Value() (driver.Value, error) { +// return ULID(id).String(), nil +// } +// +// // Example usage. +// db.Exec("...", stringValuer(id)) +func (id ULID) Value() (driver.Value, error) { + return id.MarshalBinary() +} + +// Monotonic returns an entropy source that is guaranteed to yield +// strictly increasing entropy bytes for the same ULID timestamp. +// On conflicts, the previous ULID entropy is incremented with a +// random number between 1 and `inc` (inclusive). +// +// The provided entropy source must actually yield random bytes or else +// monotonic reads are not guaranteed to terminate, since there isn't +// enough randomness to compute an increment number. +// +// When `inc == 0`, it'll be set to a secure default of `math.MaxUint32`. +// The lower the value of `inc`, the easier the next ULID within the +// same millisecond is to guess. If your code depends on ULIDs having +// secure entropy bytes, then don't go under this default unless you know +// what you're doing. +// +// The returned io.Reader isn't safe for concurrent use. +func Monotonic(entropy io.Reader, inc uint64) io.Reader { + m := monotonic{ + Reader: bufio.NewReader(entropy), + inc: inc, + } + + if m.inc == 0 { + m.inc = math.MaxUint32 + } + + if rng, ok := entropy.(*rand.Rand); ok { + m.rng = rng + } + + return &m +} + +type monotonic struct { + io.Reader + ms uint64 + inc uint64 + entropy uint80 + rand [8]byte + rng *rand.Rand +} + +func (m *monotonic) MonotonicRead(ms uint64, entropy []byte) (err error) { + if !m.entropy.IsZero() && m.ms == ms { + err = m.increment() + m.entropy.AppendTo(entropy) + } else if _, err = io.ReadFull(m.Reader, entropy); err == nil { + m.ms = ms + m.entropy.SetBytes(entropy) + } + return err +} + +// increment the previous entropy number with a random number +// of up to m.inc (inclusive). +func (m *monotonic) increment() error { + if inc, err := m.random(); err != nil { + return err + } else if m.entropy.Add(inc) { + return ErrMonotonicOverflow + } + return nil +} + +// random returns a uniform random value in [1, m.inc), reading entropy +// from m.Reader. When m.inc == 0 || m.inc == 1, it returns 1. +// Adapted from: https://golang.org/pkg/crypto/rand/#Int +func (m *monotonic) random() (inc uint64, err error) { + if m.inc <= 1 { + return 1, nil + } + + // Fast path for using a underlying rand.Rand directly. + if m.rng != nil { + // Range: [1, m.inc) + return 1 + uint64(m.rng.Int63n(int64(m.inc))), nil + } + + // bitLen is the maximum bit length needed to encode a value < m.inc. + bitLen := bits.Len64(m.inc) + + // byteLen is the maximum byte length needed to encode a value < m.inc. + byteLen := uint(bitLen+7) / 8 + + // msbitLen is the number of bits in the most significant byte of m.inc-1. + msbitLen := uint(bitLen % 8) + if msbitLen == 0 { + msbitLen = 8 + } + + for inc == 0 || inc >= m.inc { + if _, err = io.ReadFull(m.Reader, m.rand[:byteLen]); err != nil { + return 0, err + } + + // Clear bits in the first byte to increase the probability + // that the candidate is < m.inc. + m.rand[0] &= uint8(int(1< 0, segment size is WALSegmentSize. + WALSegmentSize int + + // WALCompression will turn on Snappy compression for records on the WAL. + WALCompression bool + + // StripeSize is the size (power of 2) in entries of the series hash map. Reducing the size will save memory but impact performance. + StripeSize int + + // TruncateFrequency determines how frequently to truncate data from the WAL. + TruncateFrequency time.Duration + + // Shortest and longest amount of time data can exist in the WAL before being + // deleted. + MinWALTime, MaxWALTime int64 + + // NoLockfile disables creation and consideration of a lock file. + NoLockfile bool +} + +// DefaultOptions used for the WAL storage. They are sane for setups using +// millisecond-precision timestamps. +func DefaultOptions() *Options { + return &Options{ + WALSegmentSize: wal.DefaultSegmentSize, + WALCompression: false, + StripeSize: tsdb.DefaultStripeSize, + TruncateFrequency: DefaultTruncateFrequency, + MinWALTime: DefaultMinWALTime, + MaxWALTime: DefaultMaxWALTime, + NoLockfile: false, + } +} + +type dbMetrics struct { + r prometheus.Registerer + + numActiveSeries prometheus.Gauge + numWALSeriesPendingDeletion prometheus.Gauge + totalAppendedSamples prometheus.Counter + totalAppendedExemplars prometheus.Counter + totalOutOfOrderSamples prometheus.Counter + walTruncateDuration prometheus.Summary + walCorruptionsTotal prometheus.Counter + walTotalReplayDuration prometheus.Gauge + checkpointDeleteFail prometheus.Counter + checkpointDeleteTotal prometheus.Counter + checkpointCreationFail prometheus.Counter + checkpointCreationTotal prometheus.Counter +} + +func newDBMetrics(r prometheus.Registerer) *dbMetrics { + m := dbMetrics{r: r} + m.numActiveSeries = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_agent_active_series", + Help: "Number of active series being tracked by the WAL storage", + }) + + m.numWALSeriesPendingDeletion = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_agent_deleted_series", + Help: "Number of series pending deletion from the WAL", + }) + + m.totalAppendedSamples = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_samples_appended_total", + Help: "Total number of samples appended to the storage", + }) + + m.totalAppendedExemplars = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_exemplars_appended_total", + Help: "Total number of exemplars appended to the storage", + }) + + m.totalOutOfOrderSamples = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_out_of_order_samples_total", + Help: "Total number of out of order samples ingestion failed attempts.", + }) + + m.walTruncateDuration = prometheus.NewSummary(prometheus.SummaryOpts{ + Name: "prometheus_agent_truncate_duration_seconds", + Help: "Duration of WAL truncation.", + }) + + m.walCorruptionsTotal = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_corruptions_total", + Help: "Total number of WAL corruptions.", + }) + + m.walTotalReplayDuration = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_agent_data_replay_duration_seconds", + Help: "Time taken to replay the data on disk.", + }) + + m.checkpointDeleteFail = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_checkpoint_deletions_failed_total", + Help: "Total number of checkpoint deletions that failed.", + }) + + m.checkpointDeleteTotal = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_checkpoint_deletions_total", + Help: "Total number of checkpoint deletions attempted.", + }) + + m.checkpointCreationFail = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_checkpoint_creations_failed_total", + Help: "Total number of checkpoint creations that failed.", + }) + + m.checkpointCreationTotal = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_agent_checkpoint_creations_total", + Help: "Total number of checkpoint creations attempted.", + }) + + if r != nil { + r.MustRegister( + m.numActiveSeries, + m.numWALSeriesPendingDeletion, + m.totalAppendedSamples, + m.totalAppendedExemplars, + m.totalOutOfOrderSamples, + m.walTruncateDuration, + m.walCorruptionsTotal, + m.walTotalReplayDuration, + m.checkpointDeleteFail, + m.checkpointDeleteTotal, + m.checkpointCreationFail, + m.checkpointCreationTotal, + ) + } + + return &m +} + +func (m *dbMetrics) Unregister() { + if m.r == nil { + return + } + cs := []prometheus.Collector{ + m.numActiveSeries, + m.numWALSeriesPendingDeletion, + m.totalAppendedSamples, + m.totalAppendedExemplars, + m.totalOutOfOrderSamples, + m.walTruncateDuration, + m.walCorruptionsTotal, + m.walTotalReplayDuration, + m.checkpointDeleteFail, + m.checkpointDeleteTotal, + m.checkpointCreationFail, + m.checkpointCreationTotal, + } + for _, c := range cs { + m.r.Unregister(c) + } +} + +// DB represents a WAL-only storage. It implements storage.DB. +type DB struct { + mtx sync.RWMutex + logger log.Logger + opts *Options + rs *remote.Storage + + wal *wal.WAL + locker *tsdbutil.DirLocker + + appenderPool sync.Pool + bufPool sync.Pool + + nextRef *atomic.Uint64 + series *stripeSeries + // deleted is a map of (ref IDs that should be deleted from WAL) to (the WAL segment they + // must be kept around to). + deleted map[chunks.HeadSeriesRef]int + + donec chan struct{} + stopc chan struct{} + + metrics *dbMetrics +} + +// Open returns a new agent.DB in the given directory. +func Open(l log.Logger, reg prometheus.Registerer, rs *remote.Storage, dir string, opts *Options) (*DB, error) { + opts = validateOptions(opts) + + locker, err := tsdbutil.NewDirLocker(dir, "agent", l, reg) + if err != nil { + return nil, err + } + if !opts.NoLockfile { + if err := locker.Lock(); err != nil { + return nil, err + } + } + + // remote_write expects WAL to be stored in a "wal" subdirectory of the main storage. + dir = filepath.Join(dir, "wal") + + w, err := wal.NewSize(l, reg, dir, opts.WALSegmentSize, opts.WALCompression) + if err != nil { + return nil, errors.Wrap(err, "creating WAL") + } + + db := &DB{ + logger: l, + opts: opts, + rs: rs, + + wal: w, + locker: locker, + + nextRef: atomic.NewUint64(0), + series: newStripeSeries(opts.StripeSize), + deleted: make(map[chunks.HeadSeriesRef]int), + + donec: make(chan struct{}), + stopc: make(chan struct{}), + + metrics: newDBMetrics(reg), + } + + db.bufPool.New = func() interface{} { + return make([]byte, 0, 1024) + } + + db.appenderPool.New = func() interface{} { + return &appender{ + DB: db, + pendingSeries: make([]record.RefSeries, 0, 100), + pendingSamples: make([]record.RefSample, 0, 100), + pendingExamplars: make([]record.RefExemplar, 0, 10), + } + } + + if err := db.replayWAL(); err != nil { + level.Warn(db.logger).Log("msg", "encountered WAL read error, attempting repair", "err", err) + if err := w.Repair(err); err != nil { + return nil, errors.Wrap(err, "repair corrupted WAL") + } + } + + go db.run() + return db, nil +} + +func validateOptions(opts *Options) *Options { + if opts == nil { + opts = DefaultOptions() + } + if opts.WALSegmentSize <= 0 { + opts.WALSegmentSize = wal.DefaultSegmentSize + } + + // Revert Stripesize to DefaultStripsize if Stripsize is either 0 or not a power of 2. + if opts.StripeSize <= 0 || ((opts.StripeSize & (opts.StripeSize - 1)) != 0) { + opts.StripeSize = tsdb.DefaultStripeSize + } + if opts.TruncateFrequency <= 0 { + opts.TruncateFrequency = DefaultTruncateFrequency + } + if opts.MinWALTime <= 0 { + opts.MinWALTime = 0 + } + if opts.MaxWALTime <= 0 { + opts.MaxWALTime = DefaultMaxWALTime + } + + if t := int64(opts.TruncateFrequency * time.Hour / time.Millisecond); opts.MaxWALTime < t { + opts.MaxWALTime = t + } + return opts +} + +func (db *DB) replayWAL() error { + level.Info(db.logger).Log("msg", "replaying WAL, this may take a while", "dir", db.wal.Dir()) + start := time.Now() + + dir, startFrom, err := wal.LastCheckpoint(db.wal.Dir()) + if err != nil && err != record.ErrNotFound { + return errors.Wrap(err, "find last checkpoint") + } + + multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{} + + if err == nil { + sr, err := wal.NewSegmentsReader(dir) + if err != nil { + return errors.Wrap(err, "open checkpoint") + } + defer func() { + if err := sr.Close(); err != nil { + level.Warn(db.logger).Log("msg", "error while closing the wal segments reader", "err", err) + } + }() + + // A corrupted checkpoint is a hard error for now and requires user + // intervention. There's likely little data that can be recovered anyway. + if err := db.loadWAL(wal.NewReader(sr), multiRef); err != nil { + return errors.Wrap(err, "backfill checkpoint") + } + startFrom++ + level.Info(db.logger).Log("msg", "WAL checkpoint loaded") + } + + // Find the last segment. + _, last, err := wal.Segments(db.wal.Dir()) + if err != nil { + return errors.Wrap(err, "finding WAL segments") + } + + // Backfil segments from the most recent checkpoint onwards. + for i := startFrom; i <= last; i++ { + seg, err := wal.OpenReadSegment(wal.SegmentName(db.wal.Dir(), i)) + if err != nil { + return errors.Wrap(err, fmt.Sprintf("open WAL segment: %d", i)) + } + + sr := wal.NewSegmentBufReader(seg) + err = db.loadWAL(wal.NewReader(sr), multiRef) + if err := sr.Close(); err != nil { + level.Warn(db.logger).Log("msg", "error while closing the wal segments reader", "err", err) + } + if err != nil { + return err + } + level.Info(db.logger).Log("msg", "WAL segment loaded", "segment", i, "maxSegment", last) + } + + walReplayDuration := time.Since(start) + db.metrics.walTotalReplayDuration.Set(walReplayDuration.Seconds()) + + return nil +} + +func (db *DB) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef) (err error) { + var ( + dec record.Decoder + lastRef chunks.HeadSeriesRef + + decoded = make(chan interface{}, 10) + errCh = make(chan error, 1) + seriesPool = sync.Pool{ + New: func() interface{} { + return []record.RefSeries{} + }, + } + samplesPool = sync.Pool{ + New: func() interface{} { + return []record.RefSample{} + }, + } + ) + + go func() { + defer close(decoded) + for r.Next() { + rec := r.Record() + switch dec.Type(rec) { + case record.Series: + series := seriesPool.Get().([]record.RefSeries)[:0] + series, err = dec.Series(rec, series) + if err != nil { + errCh <- &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode series"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- series + case record.Samples: + samples := samplesPool.Get().([]record.RefSample)[:0] + samples, err = dec.Samples(rec, samples) + if err != nil { + errCh <- &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode samples"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- samples + case record.Tombstones, record.Exemplars: + // We don't care about tombstones or exemplars during replay. + continue + default: + errCh <- &wal.CorruptionErr{ + Err: errors.Errorf("invalid record type %v", dec.Type(rec)), + Segment: r.Segment(), + Offset: r.Offset(), + } + } + } + }() + + var nonExistentSeriesRefs atomic.Uint64 + + for d := range decoded { + switch v := d.(type) { + case []record.RefSeries: + for _, entry := range v { + // If this is a new series, create it in memory. If we never read in a + // sample for this series, its timestamp will remain at 0 and it will + // be deleted at the next GC. + if db.series.GetByID(entry.Ref) == nil { + series := &memSeries{ref: entry.Ref, lset: entry.Labels, lastTs: 0} + db.series.Set(entry.Labels.Hash(), series) + multiRef[entry.Ref] = series.ref + db.metrics.numActiveSeries.Inc() + if entry.Ref > lastRef { + lastRef = entry.Ref + } + } + } + + //nolint:staticcheck + seriesPool.Put(v) + case []record.RefSample: + for _, entry := range v { + // Update the lastTs for the series based + ref, ok := multiRef[entry.Ref] + if !ok { + nonExistentSeriesRefs.Inc() + continue + } + series := db.series.GetByID(ref) + if entry.T > series.lastTs { + series.lastTs = entry.T + } + } + + //nolint:staticcheck + samplesPool.Put(v) + default: + panic(fmt.Errorf("unexpected decoded type: %T", d)) + } + } + + if v := nonExistentSeriesRefs.Load(); v > 0 { + level.Warn(db.logger).Log("msg", "found sample referencing non-existing series", "skipped_series", v) + } + + db.nextRef.Store(uint64(lastRef)) + + select { + case err := <-errCh: + return err + default: + if r.Err() != nil { + return errors.Wrap(r.Err(), "read records") + } + return nil + } +} + +func (db *DB) run() { + defer close(db.donec) + +Loop: + for { + select { + case <-db.stopc: + break Loop + case <-time.After(db.opts.TruncateFrequency): + // The timestamp ts is used to determine which series are not receiving + // samples and may be deleted from the WAL. Their most recent append + // timestamp is compared to ts, and if that timestamp is older then ts, + // they are considered inactive and may be deleted. + // + // Subtracting a duration from ts will add a buffer for when series are + // considered inactive and safe for deletion. + ts := db.rs.LowestSentTimestamp() - db.opts.MinWALTime + if ts < 0 { + ts = 0 + } + + // Network issues can prevent the result of getRemoteWriteTimestamp from + // changing. We don't want data in the WAL to grow forever, so we set a cap + // on the maximum age data can be. If our ts is older than this cutoff point, + // we'll shift it forward to start deleting very stale data. + if maxTS := timestamp.FromTime(time.Now()) - db.opts.MaxWALTime; ts < maxTS { + ts = maxTS + } + + level.Debug(db.logger).Log("msg", "truncating the WAL", "ts", ts) + if err := db.truncate(ts); err != nil { + level.Warn(db.logger).Log("msg", "failed to truncate WAL", "err", err) + } + } + } +} + +func (db *DB) truncate(mint int64) error { + db.mtx.RLock() + defer db.mtx.RUnlock() + + start := time.Now() + + db.gc(mint) + level.Info(db.logger).Log("msg", "series GC completed", "duration", time.Since(start)) + + first, last, err := wal.Segments(db.wal.Dir()) + if err != nil { + return errors.Wrap(err, "get segment range") + } + + // Start a new segment so low ingestion volume instances don't have more WAL + // than needed. + err = db.wal.NextSegment() + if err != nil { + return errors.Wrap(err, "next segment") + } + + last-- // Never consider most recent segment for checkpoint + if last < 0 { + return nil // no segments yet + } + + // The lower two-thirds of segments should contain mostly obsolete samples. + // If we have less than two segments, it's not worth checkpointing yet. + last = first + (last-first)*2/3 + if last <= first { + return nil + } + + keep := func(id chunks.HeadSeriesRef) bool { + if db.series.GetByID(id) != nil { + return true + } + + seg, ok := db.deleted[id] + return ok && seg >= first + } + + db.metrics.checkpointCreationTotal.Inc() + + if _, err = wal.Checkpoint(db.logger, db.wal, first, last, keep, mint); err != nil { + db.metrics.checkpointCreationFail.Inc() + if _, ok := errors.Cause(err).(*wal.CorruptionErr); ok { + db.metrics.walCorruptionsTotal.Inc() + } + return errors.Wrap(err, "create checkpoint") + } + if err := db.wal.Truncate(last + 1); err != nil { + // If truncating fails, we'll just try it again at the next checkpoint. + // Leftover segments will still just be ignored in the future if there's a + // checkpoint that supersedes them. + level.Error(db.logger).Log("msg", "truncating segments failed", "err", err) + } + + // The checkpoint is written and segments before it are truncated, so we + // no longer need to track deleted series that were being kept around. + for ref, segment := range db.deleted { + if segment < first { + delete(db.deleted, ref) + } + } + db.metrics.checkpointDeleteTotal.Inc() + db.metrics.numWALSeriesPendingDeletion.Set(float64(len(db.deleted))) + + if err := wal.DeleteCheckpoints(db.wal.Dir(), last); err != nil { + // Leftover old checkpoints do not cause problems down the line beyond + // occupying disk space. They will just be ignored since a newer checkpoint + // exists. + level.Error(db.logger).Log("msg", "delete old checkpoints", "err", err) + db.metrics.checkpointDeleteFail.Inc() + } + + db.metrics.walTruncateDuration.Observe(time.Since(start).Seconds()) + + level.Info(db.logger).Log("msg", "WAL checkpoint complete", "first", first, "last", last, "duration", time.Since(start)) + return nil +} + +// gc marks ref IDs that have not received a sample since mint as deleted in +// s.deleted, along with the segment where they originally got deleted. +func (db *DB) gc(mint int64) { + deleted := db.series.GC(mint) + db.metrics.numActiveSeries.Sub(float64(len(deleted))) + + _, last, _ := wal.Segments(db.wal.Dir()) + + // We want to keep series records for any newly deleted series + // until we've passed the last recorded segment. This prevents + // the WAL having samples for series records that no longer exist. + for ref := range deleted { + db.deleted[ref] = last + } + + db.metrics.numWALSeriesPendingDeletion.Set(float64(len(db.deleted))) +} + +// StartTime implements the Storage interface. +func (db *DB) StartTime() (int64, error) { + return int64(model.Latest), nil +} + +// Querier implements the Storage interface. +func (db *DB) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + return nil, ErrUnsupported +} + +// ChunkQuerier implements the Storage interface. +func (db *DB) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + return nil, ErrUnsupported +} + +// ExemplarQuerier implements the Storage interface. +func (db *DB) ExemplarQuerier(ctx context.Context) (storage.ExemplarQuerier, error) { + return nil, ErrUnsupported +} + +// Appender implements storage.Storage. +func (db *DB) Appender(_ context.Context) storage.Appender { + return db.appenderPool.Get().(storage.Appender) +} + +// Close implements the Storage interface. +func (db *DB) Close() error { + db.mtx.Lock() + defer db.mtx.Unlock() + + close(db.stopc) + <-db.donec + + db.metrics.Unregister() + + return tsdb_errors.NewMulti(db.locker.Release(), db.wal.Close()).Err() +} + +type appender struct { + *DB + + pendingSeries []record.RefSeries + pendingSamples []record.RefSample + pendingExamplars []record.RefExemplar + + // Pointers to the series referenced by each element of pendingSamples. + // Series lock is not held on elements. + sampleSeries []*memSeries +} + +func (a *appender) Append(ref storage.SeriesRef, l labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + // series references and chunk references are identical for agent mode. + headRef := chunks.HeadSeriesRef(ref) + + series := a.series.GetByID(headRef) + if series == nil { + // Ensure no empty or duplicate labels have gotten through. This mirrors the + // equivalent validation code in the TSDB's headAppender. + l = l.WithoutEmpty() + if len(l) == 0 { + return 0, errors.Wrap(tsdb.ErrInvalidSample, "empty labelset") + } + + if lbl, dup := l.HasDuplicateLabelNames(); dup { + return 0, errors.Wrap(tsdb.ErrInvalidSample, fmt.Sprintf(`label name "%s" is not unique`, lbl)) + } + + var created bool + series, created = a.getOrCreate(l) + if created { + a.pendingSeries = append(a.pendingSeries, record.RefSeries{ + Ref: series.ref, + Labels: l, + }) + + a.metrics.numActiveSeries.Inc() + } + } + + series.Lock() + defer series.Unlock() + + if t < series.lastTs { + a.metrics.totalOutOfOrderSamples.Inc() + return 0, storage.ErrOutOfOrderSample + } + + // NOTE: always modify pendingSamples and sampleSeries together + a.pendingSamples = append(a.pendingSamples, record.RefSample{ + Ref: series.ref, + T: t, + V: v, + }) + a.sampleSeries = append(a.sampleSeries, series) + + a.metrics.totalAppendedSamples.Inc() + return storage.SeriesRef(series.ref), nil +} + +func (a *appender) getOrCreate(l labels.Labels) (series *memSeries, created bool) { + hash := l.Hash() + + series = a.series.GetByHash(hash, l) + if series != nil { + return series, false + } + + ref := chunks.HeadSeriesRef(a.nextRef.Inc()) + series = &memSeries{ref: ref, lset: l, lastTs: math.MinInt64} + a.series.Set(hash, series) + return series, true +} + +func (a *appender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { + // series references and chunk references are identical for agent mode. + headRef := chunks.HeadSeriesRef(ref) + + s := a.series.GetByID(headRef) + if s == nil { + return 0, fmt.Errorf("unknown series ref when trying to add exemplar: %d", ref) + } + + // Ensure no empty labels have gotten through. + e.Labels = e.Labels.WithoutEmpty() + + if lbl, dup := e.Labels.HasDuplicateLabelNames(); dup { + return 0, errors.Wrap(tsdb.ErrInvalidExemplar, fmt.Sprintf(`label name "%s" is not unique`, lbl)) + } + + // Exemplar label length does not include chars involved in text rendering such as quotes + // equals sign, or commas. See definition of const ExemplarMaxLabelLength. + labelSetLen := 0 + for _, l := range e.Labels { + labelSetLen += utf8.RuneCountInString(l.Name) + labelSetLen += utf8.RuneCountInString(l.Value) + + if labelSetLen > exemplar.ExemplarMaxLabelSetLength { + return 0, storage.ErrExemplarLabelLength + } + } + + a.pendingExamplars = append(a.pendingExamplars, record.RefExemplar{ + Ref: s.ref, + T: e.Ts, + V: e.Value, + Labels: e.Labels, + }) + + return storage.SeriesRef(s.ref), nil +} + +// Commit submits the collected samples and purges the batch. +func (a *appender) Commit() error { + a.mtx.RLock() + defer a.mtx.RUnlock() + + var encoder record.Encoder + buf := a.bufPool.Get().([]byte) + + if len(a.pendingSeries) > 0 { + buf = encoder.Series(a.pendingSeries, buf) + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] + } + + if len(a.pendingSamples) > 0 { + buf = encoder.Samples(a.pendingSamples, buf) + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] + } + + if len(a.pendingExamplars) > 0 { + buf = encoder.Exemplars(a.pendingExamplars, buf) + if err := a.wal.Log(buf); err != nil { + return err + } + buf = buf[:0] + } + + var series *memSeries + for i, s := range a.pendingSamples { + series = a.sampleSeries[i] + if !series.updateTimestamp(s.T) { + a.metrics.totalOutOfOrderSamples.Inc() + } + } + + //nolint:staticcheck + a.bufPool.Put(buf) + return a.Rollback() +} + +func (a *appender) Rollback() error { + a.pendingSeries = a.pendingSeries[:0] + a.pendingSamples = a.pendingSamples[:0] + a.pendingExamplars = a.pendingExamplars[:0] + a.sampleSeries = a.sampleSeries[:0] + a.appenderPool.Put(a) + return nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/agent/series.go b/vendor/github.com/prometheus/prometheus/tsdb/agent/series.go new file mode 100644 index 00000000000..04c9f5fc39e --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/agent/series.go @@ -0,0 +1,203 @@ +// Copyright 2021 The Prometheus Authors +// 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 agent + +import ( + "sync" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb/chunks" +) + +// memSeries is a chunkless version of tsdb.memSeries. +type memSeries struct { + sync.Mutex + + ref chunks.HeadSeriesRef + lset labels.Labels + + // Last recorded timestamp. Used by Storage.gc to determine if a series is + // stale. + lastTs int64 +} + +// updateTimestamp obtains the lock on s and will attempt to update lastTs. +// fails if newTs < lastTs. +func (m *memSeries) updateTimestamp(newTs int64) bool { + m.Lock() + defer m.Unlock() + if newTs >= m.lastTs { + m.lastTs = newTs + return true + } + return false +} + +// seriesHashmap is a simple hashmap for memSeries by their label set. +// It is built on top of a regular hashmap and holds a slice of series to +// resolve hash collisions. Its methods require the hash to be submitted +// with the label set to avoid re-computing hash throughout the code. +type seriesHashmap map[uint64][]*memSeries + +func (m seriesHashmap) Get(hash uint64, lset labels.Labels) *memSeries { + for _, s := range m[hash] { + if labels.Equal(s.lset, lset) { + return s + } + } + return nil +} + +func (m seriesHashmap) Set(hash uint64, s *memSeries) { + seriesSet := m[hash] + for i, prev := range seriesSet { + if labels.Equal(prev.lset, s.lset) { + seriesSet[i] = s + return + } + } + m[hash] = append(seriesSet, s) +} + +func (m seriesHashmap) Delete(hash uint64, ref chunks.HeadSeriesRef) { + var rem []*memSeries + for _, s := range m[hash] { + if s.ref != ref { + rem = append(rem, s) + } + } + if len(rem) == 0 { + delete(m, hash) + } else { + m[hash] = rem + } +} + +// stripeSeries locks modulo ranges of IDs and hashes to reduce lock +// contention. The locks are padded to not be on the same cache line. +// Filling the padded space with the maps was profiled to be slower - +// likely due to the additional pointer dereferences. +type stripeSeries struct { + size int + series []map[chunks.HeadSeriesRef]*memSeries + hashes []seriesHashmap + locks []stripeLock + + gcMut sync.Mutex +} + +type stripeLock struct { + sync.RWMutex + // Padding to avoid multiple locks being on the same cache line. + _ [40]byte +} + +func newStripeSeries(stripeSize int) *stripeSeries { + s := &stripeSeries{ + size: stripeSize, + series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize), + hashes: make([]seriesHashmap, stripeSize), + locks: make([]stripeLock, stripeSize), + } + for i := range s.series { + s.series[i] = map[chunks.HeadSeriesRef]*memSeries{} + } + for i := range s.hashes { + s.hashes[i] = seriesHashmap{} + } + return s +} + +// GC garbage collects old series that have not received a sample after mint +// and will fully delete them. +func (s *stripeSeries) GC(mint int64) map[chunks.HeadSeriesRef]struct{} { + // NOTE(rfratto): GC will grab two locks, one for the hash and the other for + // series. It's not valid for any other function to grab both locks, + // otherwise a deadlock might occur when running GC in parallel with + // appending. + s.gcMut.Lock() + defer s.gcMut.Unlock() + + deleted := map[chunks.HeadSeriesRef]struct{}{} + for hashLock := 0; hashLock < s.size; hashLock++ { + s.locks[hashLock].Lock() + + for hash, all := range s.hashes[hashLock] { + for _, series := range all { + series.Lock() + + // Any series that has received a write since mint is still alive. + if series.lastTs >= mint { + series.Unlock() + continue + } + + // The series is stale. We need to obtain a second lock for the + // ref if it's different than the hash lock. + refLock := int(series.ref) & (s.size - 1) + if hashLock != refLock { + s.locks[refLock].Lock() + } + + deleted[series.ref] = struct{}{} + delete(s.series[refLock], series.ref) + s.hashes[hashLock].Delete(hash, series.ref) + + if hashLock != refLock { + s.locks[refLock].Unlock() + } + series.Unlock() + } + } + + s.locks[hashLock].Unlock() + } + + return deleted +} + +func (s *stripeSeries) GetByID(id chunks.HeadSeriesRef) *memSeries { + refLock := uint64(id) & uint64(s.size-1) + s.locks[refLock].RLock() + defer s.locks[refLock].RUnlock() + return s.series[refLock][id] +} + +func (s *stripeSeries) GetByHash(hash uint64, lset labels.Labels) *memSeries { + hashLock := hash & uint64(s.size-1) + + s.locks[hashLock].RLock() + defer s.locks[hashLock].RUnlock() + return s.hashes[hashLock].Get(hash, lset) +} + +func (s *stripeSeries) Set(hash uint64, series *memSeries) { + var ( + hashLock = hash & uint64(s.size-1) + refLock = uint64(series.ref) & uint64(s.size-1) + ) + + // We can't hold both locks at once otherwise we might deadlock with a + // simulatenous call to GC. + // + // We update s.series first because GC expects anything in s.hashes to + // already exist in s.series. + s.locks[refLock].Lock() + s.series[refLock][series.ref] = series + s.locks[refLock].Unlock() + + s.locks[hashLock].Lock() + s.hashes[hashLock].Set(hash, series) + s.locks[hashLock].Unlock() +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/block.go b/vendor/github.com/prometheus/prometheus/tsdb/block.go new file mode 100644 index 00000000000..e4db21f85e7 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/block.go @@ -0,0 +1,674 @@ +// Copyright 2017 The Prometheus Authors + +// 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 tsdb + +import ( + "encoding/json" + "io" + "io/ioutil" + "os" + "path/filepath" + "sort" + "sync" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/tombstones" +) + +// IndexWriter serializes the index for a block of series data. +// The methods must be called in the order they are specified in. +type IndexWriter interface { + // AddSymbols registers all string symbols that are encountered in series + // and other indices. Symbols must be added in sorted order. + AddSymbol(sym string) error + + // AddSeries populates the index writer with a series and its offsets + // of chunks that the index can reference. + // Implementations may require series to be insert in strictly increasing order by + // their labels. The reference numbers are used to resolve entries in postings lists + // that are added later. + AddSeries(ref storage.SeriesRef, l labels.Labels, chunks ...chunks.Meta) error + + // Close writes any finalization and closes the resources associated with + // the underlying writer. + Close() error +} + +// IndexReader provides reading access of serialized index data. +type IndexReader interface { + // Symbols return an iterator over sorted string symbols that may occur in + // series' labels and indices. It is not safe to use the returned strings + // beyond the lifetime of the index reader. + Symbols() index.StringIter + + // SortedLabelValues returns sorted possible label values. + SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) + + // LabelValues returns possible label values which may not be sorted. + LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) + + // Postings returns the postings list iterator for the label pairs. + // The Postings here contain the offsets to the series inside the index. + // Found IDs are not strictly required to point to a valid Series, e.g. + // during background garbage collections. Input values must be sorted. + Postings(name string, values ...string) (index.Postings, error) + + // SortedPostings returns a postings list that is reordered to be sorted + // by the label set of the underlying series. + SortedPostings(index.Postings) index.Postings + + // Series populates the given labels and chunk metas for the series identified + // by the reference. + // Returns storage.ErrNotFound if the ref does not resolve to a known series. + Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error + + // LabelNames returns all the unique label names present in the index in sorted order. + LabelNames(matchers ...*labels.Matcher) ([]string, error) + + // LabelValueFor returns label value for the given label name in the series referred to by ID. + // If the series couldn't be found or the series doesn't have the requested label a + // storage.ErrNotFound is returned as error. + LabelValueFor(id storage.SeriesRef, label string) (string, error) + + // LabelNamesFor returns all the label names for the series referred to by IDs. + // The names returned are sorted. + LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) + + // Close releases the underlying resources of the reader. + Close() error +} + +// ChunkWriter serializes a time block of chunked series data. +type ChunkWriter interface { + // WriteChunks writes several chunks. The Chunk field of the ChunkMetas + // must be populated. + // After returning successfully, the Ref fields in the ChunkMetas + // are set and can be used to retrieve the chunks from the written data. + WriteChunks(chunks ...chunks.Meta) error + + // Close writes any required finalization and closes the resources + // associated with the underlying writer. + Close() error +} + +// ChunkReader provides reading access of serialized time series data. +type ChunkReader interface { + // Chunk returns the series data chunk with the given reference. + Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) + + // Close releases all underlying resources of the reader. + Close() error +} + +// BlockReader provides reading access to a data block. +type BlockReader interface { + // Index returns an IndexReader over the block's data. + Index() (IndexReader, error) + + // Chunks returns a ChunkReader over the block's data. + Chunks() (ChunkReader, error) + + // Tombstones returns a tombstones.Reader over the block's deleted data. + Tombstones() (tombstones.Reader, error) + + // Meta provides meta information about the block reader. + Meta() BlockMeta + + // Size returns the number of bytes that the block takes up on disk. + Size() int64 +} + +// BlockMeta provides meta information about a block. +type BlockMeta struct { + // Unique identifier for the block and its contents. Changes on compaction. + ULID ulid.ULID `json:"ulid"` + + // MinTime and MaxTime specify the time range all samples + // in the block are in. + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` + + // Stats about the contents of the block. + Stats BlockStats `json:"stats,omitempty"` + + // Information on compactions the block was created from. + Compaction BlockMetaCompaction `json:"compaction"` + + // Version of the index format. + Version int `json:"version"` +} + +// BlockStats contains stats about contents of a block. +type BlockStats struct { + NumSamples uint64 `json:"numSamples,omitempty"` + NumSeries uint64 `json:"numSeries,omitempty"` + NumChunks uint64 `json:"numChunks,omitempty"` + NumTombstones uint64 `json:"numTombstones,omitempty"` +} + +// BlockDesc describes a block by ULID and time range. +type BlockDesc struct { + ULID ulid.ULID `json:"ulid"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` +} + +// BlockMetaCompaction holds information about compactions a block went through. +type BlockMetaCompaction struct { + // Maximum number of compaction cycles any source block has + // gone through. + Level int `json:"level"` + // ULIDs of all source head blocks that went into the block. + Sources []ulid.ULID `json:"sources,omitempty"` + // Indicates that during compaction it resulted in a block without any samples + // so it should be deleted on the next reloadBlocks. + Deletable bool `json:"deletable,omitempty"` + // Short descriptions of the direct blocks that were used to create + // this block. + Parents []BlockDesc `json:"parents,omitempty"` + Failed bool `json:"failed,omitempty"` +} + +const ( + indexFilename = "index" + metaFilename = "meta.json" + metaVersion1 = 1 +) + +func chunkDir(dir string) string { return filepath.Join(dir, "chunks") } + +func readMetaFile(dir string) (*BlockMeta, int64, error) { + b, err := ioutil.ReadFile(filepath.Join(dir, metaFilename)) + if err != nil { + return nil, 0, err + } + var m BlockMeta + + if err := json.Unmarshal(b, &m); err != nil { + return nil, 0, err + } + if m.Version != metaVersion1 { + return nil, 0, errors.Errorf("unexpected meta file version %d", m.Version) + } + + return &m, int64(len(b)), nil +} + +func writeMetaFile(logger log.Logger, dir string, meta *BlockMeta) (int64, error) { + meta.Version = metaVersion1 + + // Make any changes to the file appear atomic. + path := filepath.Join(dir, metaFilename) + tmp := path + ".tmp" + defer func() { + if err := os.RemoveAll(tmp); err != nil { + level.Error(logger).Log("msg", "remove tmp file", "err", err.Error()) + } + }() + + f, err := os.Create(tmp) + if err != nil { + return 0, err + } + + jsonMeta, err := json.MarshalIndent(meta, "", "\t") + if err != nil { + return 0, err + } + + n, err := f.Write(jsonMeta) + if err != nil { + return 0, tsdb_errors.NewMulti(err, f.Close()).Err() + } + + // Force the kernel to persist the file on disk to avoid data loss if the host crashes. + if err := f.Sync(); err != nil { + return 0, tsdb_errors.NewMulti(err, f.Close()).Err() + } + if err := f.Close(); err != nil { + return 0, err + } + return int64(n), fileutil.Replace(tmp, path) +} + +// Block represents a directory of time series data covering a continuous time range. +type Block struct { + mtx sync.RWMutex + closing bool + pendingReaders sync.WaitGroup + + dir string + meta BlockMeta + + // Symbol Table Size in bytes. + // We maintain this variable to avoid recalculation every time. + symbolTableSize uint64 + + chunkr ChunkReader + indexr IndexReader + tombstones tombstones.Reader + + logger log.Logger + + numBytesChunks int64 + numBytesIndex int64 + numBytesTombstone int64 + numBytesMeta int64 +} + +// OpenBlock opens the block in the directory. It can be passed a chunk pool, which is used +// to instantiate chunk structs. +func OpenBlock(logger log.Logger, dir string, pool chunkenc.Pool) (pb *Block, err error) { + if logger == nil { + logger = log.NewNopLogger() + } + var closers []io.Closer + defer func() { + if err != nil { + err = tsdb_errors.NewMulti(err, tsdb_errors.CloseAll(closers)).Err() + } + }() + meta, sizeMeta, err := readMetaFile(dir) + if err != nil { + return nil, err + } + + cr, err := chunks.NewDirReader(chunkDir(dir), pool) + if err != nil { + return nil, err + } + closers = append(closers, cr) + + ir, err := index.NewFileReader(filepath.Join(dir, indexFilename)) + if err != nil { + return nil, err + } + closers = append(closers, ir) + + tr, sizeTomb, err := tombstones.ReadTombstones(dir) + if err != nil { + return nil, err + } + closers = append(closers, tr) + + pb = &Block{ + dir: dir, + meta: *meta, + chunkr: cr, + indexr: ir, + tombstones: tr, + symbolTableSize: ir.SymbolTableSize(), + logger: logger, + numBytesChunks: cr.Size(), + numBytesIndex: ir.Size(), + numBytesTombstone: sizeTomb, + numBytesMeta: sizeMeta, + } + return pb, nil +} + +// Close closes the on-disk block. It blocks as long as there are readers reading from the block. +func (pb *Block) Close() error { + pb.mtx.Lock() + pb.closing = true + pb.mtx.Unlock() + + pb.pendingReaders.Wait() + + return tsdb_errors.NewMulti( + pb.chunkr.Close(), + pb.indexr.Close(), + pb.tombstones.Close(), + ).Err() +} + +func (pb *Block) String() string { + return pb.meta.ULID.String() +} + +// Dir returns the directory of the block. +func (pb *Block) Dir() string { return pb.dir } + +// Meta returns meta information about the block. +func (pb *Block) Meta() BlockMeta { return pb.meta } + +// MinTime returns the min time of the meta. +func (pb *Block) MinTime() int64 { return pb.meta.MinTime } + +// MaxTime returns the max time of the meta. +func (pb *Block) MaxTime() int64 { return pb.meta.MaxTime } + +// Size returns the number of bytes that the block takes up. +func (pb *Block) Size() int64 { + return pb.numBytesChunks + pb.numBytesIndex + pb.numBytesTombstone + pb.numBytesMeta +} + +// ErrClosing is returned when a block is in the process of being closed. +var ErrClosing = errors.New("block is closing") + +func (pb *Block) startRead() error { + pb.mtx.RLock() + defer pb.mtx.RUnlock() + + if pb.closing { + return ErrClosing + } + pb.pendingReaders.Add(1) + return nil +} + +// Index returns a new IndexReader against the block data. +func (pb *Block) Index() (IndexReader, error) { + if err := pb.startRead(); err != nil { + return nil, err + } + return blockIndexReader{ir: pb.indexr, b: pb}, nil +} + +// Chunks returns a new ChunkReader against the block data. +func (pb *Block) Chunks() (ChunkReader, error) { + if err := pb.startRead(); err != nil { + return nil, err + } + return blockChunkReader{ChunkReader: pb.chunkr, b: pb}, nil +} + +// Tombstones returns a new TombstoneReader against the block data. +func (pb *Block) Tombstones() (tombstones.Reader, error) { + if err := pb.startRead(); err != nil { + return nil, err + } + return blockTombstoneReader{Reader: pb.tombstones, b: pb}, nil +} + +// GetSymbolTableSize returns the Symbol Table Size in the index of this block. +func (pb *Block) GetSymbolTableSize() uint64 { + return pb.symbolTableSize +} + +func (pb *Block) setCompactionFailed() error { + pb.meta.Compaction.Failed = true + n, err := writeMetaFile(pb.logger, pb.dir, &pb.meta) + if err != nil { + return err + } + pb.numBytesMeta = n + return nil +} + +type blockIndexReader struct { + ir IndexReader + b *Block +} + +func (r blockIndexReader) Symbols() index.StringIter { + return r.ir.Symbols() +} + +func (r blockIndexReader) SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + var st []string + var err error + + if len(matchers) == 0 { + st, err = r.ir.SortedLabelValues(name) + } else { + st, err = r.LabelValues(name, matchers...) + if err == nil { + sort.Strings(st) + } + } + + return st, errors.Wrapf(err, "block: %s", r.b.Meta().ULID) +} + +func (r blockIndexReader) LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + if len(matchers) == 0 { + st, err := r.ir.LabelValues(name) + return st, errors.Wrapf(err, "block: %s", r.b.Meta().ULID) + } + + return labelValuesWithMatchers(r.ir, name, matchers...) +} + +func (r blockIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { + if len(matchers) == 0 { + return r.b.LabelNames() + } + + return labelNamesWithMatchers(r.ir, matchers...) +} + +func (r blockIndexReader) Postings(name string, values ...string) (index.Postings, error) { + p, err := r.ir.Postings(name, values...) + if err != nil { + return p, errors.Wrapf(err, "block: %s", r.b.Meta().ULID) + } + return p, nil +} + +func (r blockIndexReader) SortedPostings(p index.Postings) index.Postings { + return r.ir.SortedPostings(p) +} + +func (r blockIndexReader) Series(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error { + if err := r.ir.Series(ref, lset, chks); err != nil { + return errors.Wrapf(err, "block: %s", r.b.Meta().ULID) + } + return nil +} + +func (r blockIndexReader) Close() error { + r.b.pendingReaders.Done() + return nil +} + +// LabelValueFor returns label value for the given label name in the series referred to by ID. +func (r blockIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { + return r.ir.LabelValueFor(id, label) +} + +// LabelNamesFor returns all the label names for the series referred to by IDs. +// The names returned are sorted. +func (r blockIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { + return r.ir.LabelNamesFor(ids...) +} + +type blockTombstoneReader struct { + tombstones.Reader + b *Block +} + +func (r blockTombstoneReader) Close() error { + r.b.pendingReaders.Done() + return nil +} + +type blockChunkReader struct { + ChunkReader + b *Block +} + +func (r blockChunkReader) Close() error { + r.b.pendingReaders.Done() + return nil +} + +// Delete matching series between mint and maxt in the block. +func (pb *Block) Delete(mint, maxt int64, ms ...*labels.Matcher) error { + pb.mtx.Lock() + defer pb.mtx.Unlock() + + if pb.closing { + return ErrClosing + } + + p, err := PostingsForMatchers(pb.indexr, ms...) + if err != nil { + return errors.Wrap(err, "select series") + } + + ir := pb.indexr + + // Choose only valid postings which have chunks in the time-range. + stones := tombstones.NewMemTombstones() + + var lset labels.Labels + var chks []chunks.Meta + +Outer: + for p.Next() { + err := ir.Series(p.At(), &lset, &chks) + if err != nil { + return err + } + + for _, chk := range chks { + if chk.OverlapsClosedInterval(mint, maxt) { + // Delete only until the current values and not beyond. + tmin, tmax := clampInterval(mint, maxt, chks[0].MinTime, chks[len(chks)-1].MaxTime) + stones.AddInterval(p.At(), tombstones.Interval{Mint: tmin, Maxt: tmax}) + continue Outer + } + } + } + + if p.Err() != nil { + return p.Err() + } + + err = pb.tombstones.Iter(func(id storage.SeriesRef, ivs tombstones.Intervals) error { + for _, iv := range ivs { + stones.AddInterval(id, iv) + } + return nil + }) + if err != nil { + return err + } + pb.tombstones = stones + pb.meta.Stats.NumTombstones = pb.tombstones.Total() + + n, err := tombstones.WriteFile(pb.logger, pb.dir, pb.tombstones) + if err != nil { + return err + } + pb.numBytesTombstone = n + n, err = writeMetaFile(pb.logger, pb.dir, &pb.meta) + if err != nil { + return err + } + pb.numBytesMeta = n + return nil +} + +// CleanTombstones will remove the tombstones and rewrite the block (only if there are any tombstones). +// If there was a rewrite, then it returns the ULID of the new block written, else nil. +// If the resultant block is empty (tombstones covered the whole block), then it deletes the new block and return nil UID. +// It returns a boolean indicating if the parent block can be deleted safely of not. +func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, bool, error) { + numStones := 0 + + if err := pb.tombstones.Iter(func(id storage.SeriesRef, ivs tombstones.Intervals) error { + numStones += len(ivs) + return nil + }); err != nil { + // This should never happen, as the iteration function only returns nil. + panic(err) + } + if numStones == 0 { + return nil, false, nil + } + + meta := pb.Meta() + uid, err := c.Write(dest, pb, pb.meta.MinTime, pb.meta.MaxTime, &meta) + if err != nil { + return nil, false, err + } + + return &uid, true, nil +} + +// Snapshot creates snapshot of the block into dir. +func (pb *Block) Snapshot(dir string) error { + blockDir := filepath.Join(dir, pb.meta.ULID.String()) + if err := os.MkdirAll(blockDir, 0o777); err != nil { + return errors.Wrap(err, "create snapshot block dir") + } + + chunksDir := chunkDir(blockDir) + if err := os.MkdirAll(chunksDir, 0o777); err != nil { + return errors.Wrap(err, "create snapshot chunk dir") + } + + // Hardlink meta, index and tombstones + for _, fname := range []string{ + metaFilename, + indexFilename, + tombstones.TombstonesFilename, + } { + if err := os.Link(filepath.Join(pb.dir, fname), filepath.Join(blockDir, fname)); err != nil { + return errors.Wrapf(err, "create snapshot %s", fname) + } + } + + // Hardlink the chunks + curChunkDir := chunkDir(pb.dir) + files, err := ioutil.ReadDir(curChunkDir) + if err != nil { + return errors.Wrap(err, "ReadDir the current chunk dir") + } + + for _, f := range files { + err := os.Link(filepath.Join(curChunkDir, f.Name()), filepath.Join(chunksDir, f.Name())) + if err != nil { + return errors.Wrap(err, "hardlink a chunk") + } + } + + return nil +} + +// OverlapsClosedInterval returns true if the block overlaps [mint, maxt]. +func (pb *Block) OverlapsClosedInterval(mint, maxt int64) bool { + // The block itself is a half-open interval + // [pb.meta.MinTime, pb.meta.MaxTime). + return pb.meta.MinTime <= maxt && mint < pb.meta.MaxTime +} + +// LabelNames returns all the unique label names present in the Block in sorted order. +func (pb *Block) LabelNames() ([]string, error) { + return pb.indexr.LabelNames() +} + +func clampInterval(a, b, mint, maxt int64) (int64, int64) { + if a < mint { + a = mint + } + if b > maxt { + b = maxt + } + return a, b +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/blockwriter.go b/vendor/github.com/prometheus/prometheus/tsdb/blockwriter.go new file mode 100644 index 00000000000..064cd01c7e2 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/blockwriter.go @@ -0,0 +1,122 @@ +// Copyright 2020 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "io/ioutil" + "math" + "os" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/model/timestamp" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" +) + +// BlockWriter is a block writer that allows appending and flushing series to disk. +type BlockWriter struct { + logger log.Logger + destinationDir string + + head *Head + blockSize int64 // in ms + chunkDir string +} + +// ErrNoSeriesAppended is returned if the series count is zero while flushing blocks. +var ErrNoSeriesAppended error = errors.New("no series appended, aborting") + +// NewBlockWriter create a new block writer. +// +// The returned writer accumulates all the series in the Head block until `Flush` is called. +// +// Note that the writer will not check if the target directory exists or +// contains anything at all. It is the caller's responsibility to +// ensure that the resulting blocks do not overlap etc. +// Writer ensures the block flush is atomic (via rename). +func NewBlockWriter(logger log.Logger, dir string, blockSize int64) (*BlockWriter, error) { + w := &BlockWriter{ + logger: logger, + destinationDir: dir, + blockSize: blockSize, + } + if err := w.initHead(); err != nil { + return nil, err + } + return w, nil +} + +// initHead creates and initialises a new TSDB head. +func (w *BlockWriter) initHead() error { + chunkDir, err := ioutil.TempDir(os.TempDir(), "head") + if err != nil { + return errors.Wrap(err, "create temp dir") + } + w.chunkDir = chunkDir + opts := DefaultHeadOptions() + opts.ChunkRange = w.blockSize + opts.ChunkDirRoot = w.chunkDir + h, err := NewHead(nil, w.logger, nil, opts, NewHeadStats()) + if err != nil { + return errors.Wrap(err, "tsdb.NewHead") + } + + w.head = h + return w.head.Init(math.MinInt64) +} + +// Appender returns a new appender on the database. +// Appender can't be called concurrently. However, the returned Appender can safely be used concurrently. +func (w *BlockWriter) Appender(ctx context.Context) storage.Appender { + return w.head.Appender(ctx) +} + +// Flush implements the Writer interface. This is where actual block writing +// happens. After flush completes, no writes can be done. +func (w *BlockWriter) Flush(ctx context.Context) (ulid.ULID, error) { + mint := w.head.MinTime() + // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). + // Because of this block intervals are always +1 than the total samples it includes. + maxt := w.head.MaxTime() + 1 + level.Info(w.logger).Log("msg", "flushing", "series_count", w.head.NumSeries(), "mint", timestamp.Time(mint), "maxt", timestamp.Time(maxt)) + + compactor, err := NewLeveledCompactor(ctx, + nil, + w.logger, + []int64{w.blockSize}, + chunkenc.NewPool(), nil) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "create leveled compactor") + } + id, err := compactor.Write(w.destinationDir, w.head, mint, maxt, nil) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "compactor write") + } + + return id, nil +} + +func (w *BlockWriter) Close() error { + defer func() { + if err := os.RemoveAll(w.chunkDir); err != nil { + level.Error(w.logger).Log("msg", "error in deleting BlockWriter files", "err", err) + } + }() + return w.head.Close() +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/compact.go b/vendor/github.com/prometheus/prometheus/tsdb/compact.go new file mode 100644 index 00000000000..1d7e93f7f06 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/compact.go @@ -0,0 +1,809 @@ +// Copyright 2017 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "crypto/rand" + "fmt" + "io" + "os" + "path/filepath" + "sort" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/tombstones" +) + +// ExponentialBlockRanges returns the time ranges based on the stepSize. +func ExponentialBlockRanges(minSize int64, steps, stepSize int) []int64 { + ranges := make([]int64, 0, steps) + curRange := minSize + for i := 0; i < steps; i++ { + ranges = append(ranges, curRange) + curRange = curRange * int64(stepSize) + } + + return ranges +} + +// Compactor provides compaction against an underlying storage +// of time series data. +type Compactor interface { + // Plan returns a set of directories that can be compacted concurrently. + // The directories can be overlapping. + // Results returned when compactions are in progress are undefined. + Plan(dir string) ([]string, error) + + // Write persists a Block into a directory. + // No Block is written when resulting Block has 0 samples, and returns empty ulid.ULID{}. + Write(dest string, b BlockReader, mint, maxt int64, parent *BlockMeta) (ulid.ULID, error) + + // Compact runs compaction against the provided directories. Must + // only be called concurrently with results of Plan(). + // Can optionally pass a list of already open blocks, + // to avoid having to reopen them. + // When resulting Block has 0 samples + // * No block is written. + // * The source dirs are marked Deletable. + // * Returns empty ulid.ULID{}. + Compact(dest string, dirs []string, open []*Block) (ulid.ULID, error) +} + +// LeveledCompactor implements the Compactor interface. +type LeveledCompactor struct { + metrics *compactorMetrics + logger log.Logger + ranges []int64 + chunkPool chunkenc.Pool + ctx context.Context + maxBlockChunkSegmentSize int64 + mergeFunc storage.VerticalChunkSeriesMergeFunc +} + +type compactorMetrics struct { + ran prometheus.Counter + populatingBlocks prometheus.Gauge + overlappingBlocks prometheus.Counter + duration prometheus.Histogram + chunkSize prometheus.Histogram + chunkSamples prometheus.Histogram + chunkRange prometheus.Histogram +} + +func newCompactorMetrics(r prometheus.Registerer) *compactorMetrics { + m := &compactorMetrics{} + + m.ran = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_compactions_total", + Help: "Total number of compactions that were executed for the partition.", + }) + m.populatingBlocks = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_compaction_populating_block", + Help: "Set to 1 when a block is currently being written to the disk.", + }) + m.overlappingBlocks = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_vertical_compactions_total", + Help: "Total number of compactions done on overlapping blocks.", + }) + m.duration = prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_tsdb_compaction_duration_seconds", + Help: "Duration of compaction runs", + Buckets: prometheus.ExponentialBuckets(1, 2, 14), + }) + m.chunkSize = prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_tsdb_compaction_chunk_size_bytes", + Help: "Final size of chunks on their first compaction", + Buckets: prometheus.ExponentialBuckets(32, 1.5, 12), + }) + m.chunkSamples = prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_tsdb_compaction_chunk_samples", + Help: "Final number of samples on their first compaction", + Buckets: prometheus.ExponentialBuckets(4, 1.5, 12), + }) + m.chunkRange = prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_tsdb_compaction_chunk_range_seconds", + Help: "Final time range of chunks on their first compaction", + Buckets: prometheus.ExponentialBuckets(100, 4, 10), + }) + + if r != nil { + r.MustRegister( + m.ran, + m.populatingBlocks, + m.overlappingBlocks, + m.duration, + m.chunkRange, + m.chunkSamples, + m.chunkSize, + ) + } + return m +} + +// NewLeveledCompactor returns a LeveledCompactor. +func NewLeveledCompactor(ctx context.Context, r prometheus.Registerer, l log.Logger, ranges []int64, pool chunkenc.Pool, mergeFunc storage.VerticalChunkSeriesMergeFunc) (*LeveledCompactor, error) { + return NewLeveledCompactorWithChunkSize(ctx, r, l, ranges, pool, chunks.DefaultChunkSegmentSize, mergeFunc) +} + +func NewLeveledCompactorWithChunkSize(ctx context.Context, r prometheus.Registerer, l log.Logger, ranges []int64, pool chunkenc.Pool, maxBlockChunkSegmentSize int64, mergeFunc storage.VerticalChunkSeriesMergeFunc) (*LeveledCompactor, error) { + if len(ranges) == 0 { + return nil, errors.Errorf("at least one range must be provided") + } + if pool == nil { + pool = chunkenc.NewPool() + } + if l == nil { + l = log.NewNopLogger() + } + if mergeFunc == nil { + mergeFunc = storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge) + } + return &LeveledCompactor{ + ranges: ranges, + chunkPool: pool, + logger: l, + metrics: newCompactorMetrics(r), + ctx: ctx, + maxBlockChunkSegmentSize: maxBlockChunkSegmentSize, + mergeFunc: mergeFunc, + }, nil +} + +type dirMeta struct { + dir string + meta *BlockMeta +} + +// Plan returns a list of compactable blocks in the provided directory. +func (c *LeveledCompactor) Plan(dir string) ([]string, error) { + dirs, err := blockDirs(dir) + if err != nil { + return nil, err + } + if len(dirs) < 1 { + return nil, nil + } + + var dms []dirMeta + for _, dir := range dirs { + meta, _, err := readMetaFile(dir) + if err != nil { + return nil, err + } + dms = append(dms, dirMeta{dir, meta}) + } + return c.plan(dms) +} + +func (c *LeveledCompactor) plan(dms []dirMeta) ([]string, error) { + sort.Slice(dms, func(i, j int) bool { + return dms[i].meta.MinTime < dms[j].meta.MinTime + }) + + res := c.selectOverlappingDirs(dms) + if len(res) > 0 { + return res, nil + } + // No overlapping blocks, do compaction the usual way. + // We do not include a recently created block with max(minTime), so the block which was just created from WAL. + // This gives users a window of a full block size to piece-wise backup new data without having to care about data overlap. + dms = dms[:len(dms)-1] + + for _, dm := range c.selectDirs(dms) { + res = append(res, dm.dir) + } + if len(res) > 0 { + return res, nil + } + + // Compact any blocks with big enough time range that have >5% tombstones. + for i := len(dms) - 1; i >= 0; i-- { + meta := dms[i].meta + if meta.MaxTime-meta.MinTime < c.ranges[len(c.ranges)/2] { + // If the block is entirely deleted, then we don't care about the block being big enough. + // TODO: This is assuming single tombstone is for distinct series, which might be no true. + if meta.Stats.NumTombstones > 0 && meta.Stats.NumTombstones >= meta.Stats.NumSeries { + return []string{dms[i].dir}, nil + } + break + } + if float64(meta.Stats.NumTombstones)/float64(meta.Stats.NumSeries+1) > 0.05 { + return []string{dms[i].dir}, nil + } + } + + return nil, nil +} + +// selectDirs returns the dir metas that should be compacted into a single new block. +// If only a single block range is configured, the result is always nil. +func (c *LeveledCompactor) selectDirs(ds []dirMeta) []dirMeta { + if len(c.ranges) < 2 || len(ds) < 1 { + return nil + } + + highTime := ds[len(ds)-1].meta.MinTime + + for _, iv := range c.ranges[1:] { + parts := splitByRange(ds, iv) + if len(parts) == 0 { + continue + } + + Outer: + for _, p := range parts { + // Do not select the range if it has a block whose compaction failed. + for _, dm := range p { + if dm.meta.Compaction.Failed { + continue Outer + } + } + + mint := p[0].meta.MinTime + maxt := p[len(p)-1].meta.MaxTime + // Pick the range of blocks if it spans the full range (potentially with gaps) + // or is before the most recent block. + // This ensures we don't compact blocks prematurely when another one of the same + // size still fits in the range. + if (maxt-mint == iv || maxt <= highTime) && len(p) > 1 { + return p + } + } + } + + return nil +} + +// selectOverlappingDirs returns all dirs with overlapping time ranges. +// It expects sorted input by mint and returns the overlapping dirs in the same order as received. +func (c *LeveledCompactor) selectOverlappingDirs(ds []dirMeta) []string { + if len(ds) < 2 { + return nil + } + var overlappingDirs []string + globalMaxt := ds[0].meta.MaxTime + for i, d := range ds[1:] { + if d.meta.MinTime < globalMaxt { + if len(overlappingDirs) == 0 { // When it is the first overlap, need to add the last one as well. + overlappingDirs = append(overlappingDirs, ds[i].dir) + } + overlappingDirs = append(overlappingDirs, d.dir) + } else if len(overlappingDirs) > 0 { + break + } + if d.meta.MaxTime > globalMaxt { + globalMaxt = d.meta.MaxTime + } + } + return overlappingDirs +} + +// splitByRange splits the directories by the time range. The range sequence starts at 0. +// +// For example, if we have blocks [0-10, 10-20, 50-60, 90-100] and the split range tr is 30 +// it returns [0-10, 10-20], [50-60], [90-100]. +func splitByRange(ds []dirMeta, tr int64) [][]dirMeta { + var splitDirs [][]dirMeta + + for i := 0; i < len(ds); { + var ( + group []dirMeta + t0 int64 + m = ds[i].meta + ) + // Compute start of aligned time range of size tr closest to the current block's start. + if m.MinTime >= 0 { + t0 = tr * (m.MinTime / tr) + } else { + t0 = tr * ((m.MinTime - tr + 1) / tr) + } + // Skip blocks that don't fall into the range. This can happen via mis-alignment or + // by being the multiple of the intended range. + if m.MaxTime > t0+tr { + i++ + continue + } + + // Add all dirs to the current group that are within [t0, t0+tr]. + for ; i < len(ds); i++ { + // Either the block falls into the next range or doesn't fit at all (checked above). + if ds[i].meta.MaxTime > t0+tr { + break + } + group = append(group, ds[i]) + } + + if len(group) > 0 { + splitDirs = append(splitDirs, group) + } + } + + return splitDirs +} + +// CompactBlockMetas merges many block metas into one, combining it's source blocks together +// and adjusting compaction level. Min/Max time of result block meta covers all input blocks. +func CompactBlockMetas(uid ulid.ULID, blocks ...*BlockMeta) *BlockMeta { + res := &BlockMeta{ + ULID: uid, + } + + sources := map[ulid.ULID]struct{}{} + mint := blocks[0].MinTime + maxt := blocks[0].MaxTime + + for _, b := range blocks { + if b.MinTime < mint { + mint = b.MinTime + } + if b.MaxTime > maxt { + maxt = b.MaxTime + } + if b.Compaction.Level > res.Compaction.Level { + res.Compaction.Level = b.Compaction.Level + } + for _, s := range b.Compaction.Sources { + sources[s] = struct{}{} + } + res.Compaction.Parents = append(res.Compaction.Parents, BlockDesc{ + ULID: b.ULID, + MinTime: b.MinTime, + MaxTime: b.MaxTime, + }) + } + res.Compaction.Level++ + + for s := range sources { + res.Compaction.Sources = append(res.Compaction.Sources, s) + } + sort.Slice(res.Compaction.Sources, func(i, j int) bool { + return res.Compaction.Sources[i].Compare(res.Compaction.Sources[j]) < 0 + }) + + res.MinTime = mint + res.MaxTime = maxt + return res +} + +// Compact creates a new block in the compactor's directory from the blocks in the +// provided directories. +func (c *LeveledCompactor) Compact(dest string, dirs []string, open []*Block) (uid ulid.ULID, err error) { + var ( + blocks []BlockReader + bs []*Block + metas []*BlockMeta + uids []string + ) + start := time.Now() + + for _, d := range dirs { + meta, _, err := readMetaFile(d) + if err != nil { + return uid, err + } + + var b *Block + + // Use already open blocks if we can, to avoid + // having the index data in memory twice. + for _, o := range open { + if meta.ULID == o.Meta().ULID { + b = o + break + } + } + + if b == nil { + var err error + b, err = OpenBlock(c.logger, d, c.chunkPool) + if err != nil { + return uid, err + } + defer b.Close() + } + + metas = append(metas, meta) + blocks = append(blocks, b) + bs = append(bs, b) + uids = append(uids, meta.ULID.String()) + } + + uid = ulid.MustNew(ulid.Now(), rand.Reader) + + meta := CompactBlockMetas(uid, metas...) + err = c.write(dest, meta, blocks...) + if err == nil { + if meta.Stats.NumSamples == 0 { + for _, b := range bs { + b.meta.Compaction.Deletable = true + n, err := writeMetaFile(c.logger, b.dir, &b.meta) + if err != nil { + level.Error(c.logger).Log( + "msg", "Failed to write 'Deletable' to meta file after compaction", + "ulid", b.meta.ULID, + ) + } + b.numBytesMeta = n + } + uid = ulid.ULID{} + level.Info(c.logger).Log( + "msg", "compact blocks resulted in empty block", + "count", len(blocks), + "sources", fmt.Sprintf("%v", uids), + "duration", time.Since(start), + ) + } else { + level.Info(c.logger).Log( + "msg", "compact blocks", + "count", len(blocks), + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "ulid", meta.ULID, + "sources", fmt.Sprintf("%v", uids), + "duration", time.Since(start), + ) + } + return uid, nil + } + + errs := tsdb_errors.NewMulti(err) + if err != context.Canceled { + for _, b := range bs { + if err := b.setCompactionFailed(); err != nil { + errs.Add(errors.Wrapf(err, "setting compaction failed for block: %s", b.Dir())) + } + } + } + + return uid, errs.Err() +} + +func (c *LeveledCompactor) Write(dest string, b BlockReader, mint, maxt int64, parent *BlockMeta) (ulid.ULID, error) { + start := time.Now() + + uid := ulid.MustNew(ulid.Now(), rand.Reader) + + meta := &BlockMeta{ + ULID: uid, + MinTime: mint, + MaxTime: maxt, + } + meta.Compaction.Level = 1 + meta.Compaction.Sources = []ulid.ULID{uid} + + if parent != nil { + meta.Compaction.Parents = []BlockDesc{ + {ULID: parent.ULID, MinTime: parent.MinTime, MaxTime: parent.MaxTime}, + } + } + + err := c.write(dest, meta, b) + if err != nil { + return uid, err + } + + if meta.Stats.NumSamples == 0 { + level.Info(c.logger).Log( + "msg", "write block resulted in empty block", + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "duration", time.Since(start), + ) + return ulid.ULID{}, nil + } + + level.Info(c.logger).Log( + "msg", "write block", + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "ulid", meta.ULID, + "duration", time.Since(start), + ) + return uid, nil +} + +// instrumentedChunkWriter is used for level 1 compactions to record statistics +// about compacted chunks. +type instrumentedChunkWriter struct { + ChunkWriter + + size prometheus.Histogram + samples prometheus.Histogram + trange prometheus.Histogram +} + +func (w *instrumentedChunkWriter) WriteChunks(chunks ...chunks.Meta) error { + for _, c := range chunks { + w.size.Observe(float64(len(c.Chunk.Bytes()))) + w.samples.Observe(float64(c.Chunk.NumSamples())) + w.trange.Observe(float64(c.MaxTime - c.MinTime)) + } + return w.ChunkWriter.WriteChunks(chunks...) +} + +// write creates a new block that is the union of the provided blocks into dir. +func (c *LeveledCompactor) write(dest string, meta *BlockMeta, blocks ...BlockReader) (err error) { + dir := filepath.Join(dest, meta.ULID.String()) + tmp := dir + tmpForCreationBlockDirSuffix + var closers []io.Closer + defer func(t time.Time) { + err = tsdb_errors.NewMulti(err, tsdb_errors.CloseAll(closers)).Err() + + // RemoveAll returns no error when tmp doesn't exist so it is safe to always run it. + if err := os.RemoveAll(tmp); err != nil { + level.Error(c.logger).Log("msg", "removed tmp folder after failed compaction", "err", err.Error()) + } + c.metrics.ran.Inc() + c.metrics.duration.Observe(time.Since(t).Seconds()) + }(time.Now()) + + if err = os.RemoveAll(tmp); err != nil { + return err + } + + if err = os.MkdirAll(tmp, 0o777); err != nil { + return err + } + + // Populate chunk and index files into temporary directory with + // data of all blocks. + var chunkw ChunkWriter + + chunkw, err = chunks.NewWriterWithSegSize(chunkDir(tmp), c.maxBlockChunkSegmentSize) + if err != nil { + return errors.Wrap(err, "open chunk writer") + } + closers = append(closers, chunkw) + // Record written chunk sizes on level 1 compactions. + if meta.Compaction.Level == 1 { + chunkw = &instrumentedChunkWriter{ + ChunkWriter: chunkw, + size: c.metrics.chunkSize, + samples: c.metrics.chunkSamples, + trange: c.metrics.chunkRange, + } + } + + indexw, err := index.NewWriter(c.ctx, filepath.Join(tmp, indexFilename)) + if err != nil { + return errors.Wrap(err, "open index writer") + } + closers = append(closers, indexw) + + if err := c.populateBlock(blocks, meta, indexw, chunkw); err != nil { + return errors.Wrap(err, "populate block") + } + + select { + case <-c.ctx.Done(): + return c.ctx.Err() + default: + } + + // We are explicitly closing them here to check for error even + // though these are covered under defer. This is because in Windows, + // you cannot delete these unless they are closed and the defer is to + // make sure they are closed if the function exits due to an error above. + errs := tsdb_errors.NewMulti() + for _, w := range closers { + errs.Add(w.Close()) + } + closers = closers[:0] // Avoid closing the writers twice in the defer. + if errs.Err() != nil { + return errs.Err() + } + + // Populated block is empty, so exit early. + if meta.Stats.NumSamples == 0 { + return nil + } + + if _, err = writeMetaFile(c.logger, tmp, meta); err != nil { + return errors.Wrap(err, "write merged meta") + } + + // Create an empty tombstones file. + if _, err := tombstones.WriteFile(c.logger, tmp, tombstones.NewMemTombstones()); err != nil { + return errors.Wrap(err, "write new tombstones file") + } + + df, err := fileutil.OpenDir(tmp) + if err != nil { + return errors.Wrap(err, "open temporary block dir") + } + defer func() { + if df != nil { + df.Close() + } + }() + + if err := df.Sync(); err != nil { + return errors.Wrap(err, "sync temporary dir file") + } + + // Close temp dir before rename block dir (for windows platform). + if err = df.Close(); err != nil { + return errors.Wrap(err, "close temporary dir") + } + df = nil + + // Block successfully written, make it visible in destination dir by moving it from tmp one. + if err := fileutil.Replace(tmp, dir); err != nil { + return errors.Wrap(err, "rename block dir") + } + + return nil +} + +// populateBlock fills the index and chunk writers with new data gathered as the union +// of the provided blocks. It returns meta information for the new block. +// It expects sorted blocks input by mint. +func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta, indexw IndexWriter, chunkw ChunkWriter) (err error) { + if len(blocks) == 0 { + return errors.New("cannot populate block from no readers") + } + + var ( + sets []storage.ChunkSeriesSet + symbols index.StringIter + closers []io.Closer + overlapping bool + ) + defer func() { + errs := tsdb_errors.NewMulti(err) + if cerr := tsdb_errors.CloseAll(closers); cerr != nil { + errs.Add(errors.Wrap(cerr, "close")) + } + err = errs.Err() + c.metrics.populatingBlocks.Set(0) + }() + c.metrics.populatingBlocks.Set(1) + + globalMaxt := blocks[0].Meta().MaxTime + for i, b := range blocks { + select { + case <-c.ctx.Done(): + return c.ctx.Err() + default: + } + + if !overlapping { + if i > 0 && b.Meta().MinTime < globalMaxt { + c.metrics.overlappingBlocks.Inc() + overlapping = true + level.Info(c.logger).Log("msg", "Found overlapping blocks during compaction", "ulid", meta.ULID) + } + if b.Meta().MaxTime > globalMaxt { + globalMaxt = b.Meta().MaxTime + } + } + + indexr, err := b.Index() + if err != nil { + return errors.Wrapf(err, "open index reader for block %+v", b.Meta()) + } + closers = append(closers, indexr) + + chunkr, err := b.Chunks() + if err != nil { + return errors.Wrapf(err, "open chunk reader for block %+v", b.Meta()) + } + closers = append(closers, chunkr) + + tombsr, err := b.Tombstones() + if err != nil { + return errors.Wrapf(err, "open tombstone reader for block %+v", b.Meta()) + } + closers = append(closers, tombsr) + + k, v := index.AllPostingsKey() + all, err := indexr.Postings(k, v) + if err != nil { + return err + } + all = indexr.SortedPostings(all) + // Blocks meta is half open: [min, max), so subtract 1 to ensure we don't hold samples with exact meta.MaxTime timestamp. + sets = append(sets, newBlockChunkSeriesSet(indexr, chunkr, tombsr, all, meta.MinTime, meta.MaxTime-1, false)) + syms := indexr.Symbols() + if i == 0 { + symbols = syms + continue + } + symbols = NewMergedStringIter(symbols, syms) + } + + for symbols.Next() { + if err := indexw.AddSymbol(symbols.At()); err != nil { + return errors.Wrap(err, "add symbol") + } + } + if symbols.Err() != nil { + return errors.Wrap(symbols.Err(), "next symbol") + } + + var ( + ref = storage.SeriesRef(0) + chks []chunks.Meta + ) + + set := sets[0] + if len(sets) > 1 { + // Merge series using specified chunk series merger. + // The default one is the compacting series merger. + set = storage.NewMergeChunkSeriesSet(sets, c.mergeFunc) + } + + // Iterate over all sorted chunk series. + for set.Next() { + select { + case <-c.ctx.Done(): + return c.ctx.Err() + default: + } + s := set.At() + chksIter := s.Iterator() + chks = chks[:0] + for chksIter.Next() { + // We are not iterating in streaming way over chunk as it's more efficient to do bulk write for index and + // chunk file purposes. + chks = append(chks, chksIter.At()) + } + if chksIter.Err() != nil { + return errors.Wrap(chksIter.Err(), "chunk iter") + } + + // Skip the series with all deleted chunks. + if len(chks) == 0 { + continue + } + + if err := chunkw.WriteChunks(chks...); err != nil { + return errors.Wrap(err, "write chunks") + } + if err := indexw.AddSeries(ref, s.Labels(), chks...); err != nil { + return errors.Wrap(err, "add series") + } + + meta.Stats.NumChunks += uint64(len(chks)) + meta.Stats.NumSeries++ + for _, chk := range chks { + meta.Stats.NumSamples += uint64(chk.Chunk.NumSamples()) + } + + for _, chk := range chks { + if err := c.chunkPool.Put(chk.Chunk); err != nil { + return errors.Wrap(err, "put chunk") + } + } + ref++ + } + if set.Err() != nil { + return errors.Wrap(set.Err(), "iterate compaction set") + } + + return nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/db.go b/vendor/github.com/prometheus/prometheus/tsdb/db.go new file mode 100644 index 00000000000..c85cd84cfef --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/db.go @@ -0,0 +1,1796 @@ +// Copyright 2017 The Prometheus Authors +// 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 tsdb implements a time series storage for float64 sample data. +package tsdb + +import ( + "context" + "fmt" + "io" + "io/ioutil" + "math" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "golang.org/x/sync/errgroup" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/fileutil" + _ "github.com/prometheus/prometheus/tsdb/goversion" // Load the package into main to make sure minium Go version is met. + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/prometheus/prometheus/tsdb/wal" +) + +const ( + // Default duration of a block in milliseconds. + DefaultBlockDuration = int64(2 * time.Hour / time.Millisecond) + + // Block dir suffixes to make deletion and creation operations atomic. + // We decided to do suffixes instead of creating meta.json as last (or delete as first) one, + // because in error case you still can recover meta.json from the block content within local TSDB dir. + // TODO(bwplotka): TSDB can end up with various .tmp files (e.g meta.json.tmp, WAL or segment tmp file. Think + // about removing those too on start to save space. Currently only blocks tmp dirs are removed. + tmpForDeletionBlockDirSuffix = ".tmp-for-deletion" + tmpForCreationBlockDirSuffix = ".tmp-for-creation" + // Pre-2.21 tmp dir suffix, used in clean-up functions. + tmpLegacy = ".tmp" +) + +// ErrNotReady is returned if the underlying storage is not ready yet. +var ErrNotReady = errors.New("TSDB not ready") + +// DefaultOptions used for the DB. They are sane for setups using +// millisecond precision timestamps. +func DefaultOptions() *Options { + return &Options{ + WALSegmentSize: wal.DefaultSegmentSize, + MaxBlockChunkSegmentSize: chunks.DefaultChunkSegmentSize, + RetentionDuration: int64(15 * 24 * time.Hour / time.Millisecond), + MinBlockDuration: DefaultBlockDuration, + MaxBlockDuration: DefaultBlockDuration, + NoLockfile: false, + AllowOverlappingBlocks: false, + WALCompression: false, + StripeSize: DefaultStripeSize, + HeadChunksWriteBufferSize: chunks.DefaultWriteBufferSize, + IsolationDisabled: defaultIsolationDisabled, + HeadChunksWriteQueueSize: chunks.DefaultWriteQueueSize, + } +} + +// Options of the DB storage. +type Options struct { + // Segments (wal files) max size. + // WALSegmentSize = 0, segment size is default size. + // WALSegmentSize > 0, segment size is WALSegmentSize. + // WALSegmentSize < 0, wal is disabled. + WALSegmentSize int + + // MaxBlockChunkSegmentSize is the max size of block chunk segment files. + // MaxBlockChunkSegmentSize = 0, chunk segment size is default size. + // MaxBlockChunkSegmentSize > 0, chunk segment size is MaxBlockChunkSegmentSize. + MaxBlockChunkSegmentSize int64 + + // Duration of persisted data to keep. + // Unit agnostic as long as unit is consistent with MinBlockDuration and MaxBlockDuration. + // Typically it is in milliseconds. + RetentionDuration int64 + + // Maximum number of bytes in blocks to be retained. + // 0 or less means disabled. + // NOTE: For proper storage calculations need to consider + // the size of the WAL folder which is not added when calculating + // the current size of the database. + MaxBytes int64 + + // NoLockfile disables creation and consideration of a lock file. + NoLockfile bool + + // Overlapping blocks are allowed if AllowOverlappingBlocks is true. + // This in-turn enables vertical compaction and vertical query merge. + AllowOverlappingBlocks bool + + // WALCompression will turn on Snappy compression for records on the WAL. + WALCompression bool + + // StripeSize is the size in entries of the series hash map. Reducing the size will save memory but impact performance. + StripeSize int + + // The timestamp range of head blocks after which they get persisted. + // It's the minimum duration of any persisted block. + // Unit agnostic as long as unit is consistent with RetentionDuration and MaxBlockDuration. + // Typically it is in milliseconds. + MinBlockDuration int64 + + // The maximum timestamp range of compacted blocks. + // Unit agnostic as long as unit is consistent with MinBlockDuration and RetentionDuration. + // Typically it is in milliseconds. + MaxBlockDuration int64 + + // HeadChunksWriteBufferSize configures the write buffer size used by the head chunks mapper. + HeadChunksWriteBufferSize int + + // HeadChunksWriteQueueSize configures the size of the chunk write queue used in the head chunks mapper. + HeadChunksWriteQueueSize int + + // SeriesLifecycleCallback specifies a list of callbacks that will be called during a lifecycle of a series. + // It is always a no-op in Prometheus and mainly meant for external users who import TSDB. + SeriesLifecycleCallback SeriesLifecycleCallback + + // BlocksToDelete is a function which returns the blocks which can be deleted. + // It is always the default time and size based retention in Prometheus and + // mainly meant for external users who import TSDB. + BlocksToDelete BlocksToDeleteFunc + + // Enables the in memory exemplar storage. + EnableExemplarStorage bool + + // Enables the snapshot of in-memory chunks on shutdown. This makes restarts faster. + EnableMemorySnapshotOnShutdown bool + + // MaxExemplars sets the size, in # of exemplars stored, of the single circular buffer used to store exemplars in memory. + // See tsdb/exemplar.go, specifically the CircularExemplarStorage struct and it's constructor NewCircularExemplarStorage. + MaxExemplars int64 + + // Disables isolation between reads and in-flight appends. + IsolationDisabled bool +} + +type BlocksToDeleteFunc func(blocks []*Block) map[ulid.ULID]struct{} + +// DB handles reads and writes of time series falling into +// a hashed partition of a seriedb. +type DB struct { + dir string + locker *tsdbutil.DirLocker + + logger log.Logger + metrics *dbMetrics + opts *Options + chunkPool chunkenc.Pool + compactor Compactor + blocksToDelete BlocksToDeleteFunc + + // Mutex for that must be held when modifying the general block layout. + mtx sync.RWMutex + blocks []*Block + + head *Head + + compactc chan struct{} + donec chan struct{} + stopc chan struct{} + + // cmtx ensures that compactions and deletions don't run simultaneously. + cmtx sync.Mutex + + // autoCompactMtx ensures that no compaction gets triggered while + // changing the autoCompact var. + autoCompactMtx sync.Mutex + autoCompact bool + + // Cancel a running compaction when a shutdown is initiated. + compactCancel context.CancelFunc +} + +type dbMetrics struct { + loadedBlocks prometheus.GaugeFunc + symbolTableSize prometheus.GaugeFunc + reloads prometheus.Counter + reloadsFailed prometheus.Counter + compactionsFailed prometheus.Counter + compactionsTriggered prometheus.Counter + compactionsSkipped prometheus.Counter + sizeRetentionCount prometheus.Counter + timeRetentionCount prometheus.Counter + startTime prometheus.GaugeFunc + tombCleanTimer prometheus.Histogram + blocksBytes prometheus.Gauge + maxBytes prometheus.Gauge +} + +func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { + m := &dbMetrics{} + + m.loadedBlocks = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_blocks_loaded", + Help: "Number of currently loaded data blocks", + }, func() float64 { + db.mtx.RLock() + defer db.mtx.RUnlock() + return float64(len(db.blocks)) + }) + m.symbolTableSize = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_symbol_table_size_bytes", + Help: "Size of symbol table in memory for loaded blocks", + }, func() float64 { + db.mtx.RLock() + blocks := db.blocks[:] + db.mtx.RUnlock() + symTblSize := uint64(0) + for _, b := range blocks { + symTblSize += b.GetSymbolTableSize() + } + return float64(symTblSize) + }) + m.reloads = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_reloads_total", + Help: "Number of times the database reloaded block data from disk.", + }) + m.reloadsFailed = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_reloads_failures_total", + Help: "Number of times the database failed to reloadBlocks block data from disk.", + }) + m.compactionsTriggered = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_compactions_triggered_total", + Help: "Total number of triggered compactions for the partition.", + }) + m.compactionsFailed = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_compactions_failed_total", + Help: "Total number of compactions that failed for the partition.", + }) + m.timeRetentionCount = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_time_retentions_total", + Help: "The number of times that blocks were deleted because the maximum time limit was exceeded.", + }) + m.compactionsSkipped = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_compactions_skipped_total", + Help: "Total number of skipped compactions due to disabled auto compaction.", + }) + m.startTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_lowest_timestamp", + Help: "Lowest timestamp value stored in the database. The unit is decided by the library consumer.", + }, func() float64 { + db.mtx.RLock() + defer db.mtx.RUnlock() + if len(db.blocks) == 0 { + return float64(db.head.MinTime()) + } + return float64(db.blocks[0].meta.MinTime) + }) + m.tombCleanTimer = prometheus.NewHistogram(prometheus.HistogramOpts{ + Name: "prometheus_tsdb_tombstone_cleanup_seconds", + Help: "The time taken to recompact blocks to remove tombstones.", + }) + m.blocksBytes = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_storage_blocks_bytes", + Help: "The number of bytes that are currently used for local storage by all blocks.", + }) + m.maxBytes = prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_retention_limit_bytes", + Help: "Max number of bytes to be retained in the tsdb blocks, configured 0 means disabled", + }) + m.sizeRetentionCount = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_size_retentions_total", + Help: "The number of times that blocks were deleted because the maximum number of bytes was exceeded.", + }) + + if r != nil { + r.MustRegister( + m.loadedBlocks, + m.symbolTableSize, + m.reloads, + m.reloadsFailed, + m.compactionsFailed, + m.compactionsTriggered, + m.compactionsSkipped, + m.sizeRetentionCount, + m.timeRetentionCount, + m.startTime, + m.tombCleanTimer, + m.blocksBytes, + m.maxBytes, + ) + } + return m +} + +// DBStats contains statistics about the DB seperated by component (eg. head). +// They are available before the DB has finished initializing. +type DBStats struct { + Head *HeadStats +} + +// NewDBStats returns a new DBStats object initialized using the +// the new function from each component. +func NewDBStats() *DBStats { + return &DBStats{ + Head: NewHeadStats(), + } +} + +// ErrClosed is returned when the db is closed. +var ErrClosed = errors.New("db already closed") + +// DBReadOnly provides APIs for read only operations on a database. +// Current implementation doesn't support concurrency so +// all API calls should happen in the same go routine. +type DBReadOnly struct { + logger log.Logger + dir string + closers []io.Closer + closed chan struct{} +} + +// OpenDBReadOnly opens DB in the given directory for read only operations. +func OpenDBReadOnly(dir string, l log.Logger) (*DBReadOnly, error) { + if _, err := os.Stat(dir); err != nil { + return nil, errors.Wrap(err, "opening the db dir") + } + + if l == nil { + l = log.NewNopLogger() + } + + return &DBReadOnly{ + logger: l, + dir: dir, + closed: make(chan struct{}), + }, nil +} + +// FlushWAL creates a new block containing all data that's currently in the memory buffer/WAL. +// Samples that are in existing blocks will not be written to the new block. +// Note that if the read only database is running concurrently with a +// writable database then writing the WAL to the database directory can race. +func (db *DBReadOnly) FlushWAL(dir string) (returnErr error) { + blockReaders, err := db.Blocks() + if err != nil { + return errors.Wrap(err, "read blocks") + } + maxBlockTime := int64(math.MinInt64) + if len(blockReaders) > 0 { + maxBlockTime = blockReaders[len(blockReaders)-1].Meta().MaxTime + } + w, err := wal.Open(db.logger, filepath.Join(db.dir, "wal")) + if err != nil { + return err + } + opts := DefaultHeadOptions() + opts.ChunkDirRoot = db.dir + head, err := NewHead(nil, db.logger, w, opts, NewHeadStats()) + if err != nil { + return err + } + defer func() { + returnErr = tsdb_errors.NewMulti( + returnErr, + errors.Wrap(head.Close(), "closing Head"), + ).Err() + }() + // Set the min valid time for the ingested wal samples + // to be no lower than the maxt of the last block. + if err := head.Init(maxBlockTime); err != nil { + return errors.Wrap(err, "read WAL") + } + mint := head.MinTime() + maxt := head.MaxTime() + rh := NewRangeHead(head, mint, maxt) + compactor, err := NewLeveledCompactor( + context.Background(), + nil, + db.logger, + ExponentialBlockRanges(DefaultOptions().MinBlockDuration, 3, 5), + chunkenc.NewPool(), + nil, + ) + if err != nil { + return errors.Wrap(err, "create leveled compactor") + } + // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). + // Because of this block intervals are always +1 than the total samples it includes. + _, err = compactor.Write(dir, rh, mint, maxt+1, nil) + return errors.Wrap(err, "writing WAL") +} + +func (db *DBReadOnly) loadDataAsQueryable(maxt int64) (storage.SampleAndChunkQueryable, error) { + select { + case <-db.closed: + return nil, ErrClosed + default: + } + blockReaders, err := db.Blocks() + if err != nil { + return nil, err + } + blocks := make([]*Block, len(blockReaders)) + for i, b := range blockReaders { + b, ok := b.(*Block) + if !ok { + return nil, errors.New("unable to convert a read only block to a normal block") + } + blocks[i] = b + } + + opts := DefaultHeadOptions() + opts.ChunkDirRoot = db.dir + head, err := NewHead(nil, db.logger, nil, opts, NewHeadStats()) + if err != nil { + return nil, err + } + maxBlockTime := int64(math.MinInt64) + if len(blocks) > 0 { + maxBlockTime = blocks[len(blocks)-1].Meta().MaxTime + } + + // Also add the WAL if the current blocks don't cover the requests time range. + if maxBlockTime <= maxt { + if err := head.Close(); err != nil { + return nil, err + } + w, err := wal.Open(db.logger, filepath.Join(db.dir, "wal")) + if err != nil { + return nil, err + } + opts := DefaultHeadOptions() + opts.ChunkDirRoot = db.dir + head, err = NewHead(nil, db.logger, w, opts, NewHeadStats()) + if err != nil { + return nil, err + } + // Set the min valid time for the ingested wal samples + // to be no lower than the maxt of the last block. + if err := head.Init(maxBlockTime); err != nil { + return nil, errors.Wrap(err, "read WAL") + } + // Set the wal to nil to disable all wal operations. + // This is mainly to avoid blocking when closing the head. + head.wal = nil + } + + db.closers = append(db.closers, head) + return &DB{ + dir: db.dir, + logger: db.logger, + blocks: blocks, + head: head, + }, nil +} + +// Querier loads the blocks and wal and returns a new querier over the data partition for the given time range. +// Current implementation doesn't support multiple Queriers. +func (db *DBReadOnly) Querier(ctx context.Context, mint, maxt int64) (storage.Querier, error) { + q, err := db.loadDataAsQueryable(maxt) + if err != nil { + return nil, err + } + return q.Querier(ctx, mint, maxt) +} + +// ChunkQuerier loads blocks and the wal and returns a new chunk querier over the data partition for the given time range. +// Current implementation doesn't support multiple ChunkQueriers. +func (db *DBReadOnly) ChunkQuerier(ctx context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + q, err := db.loadDataAsQueryable(maxt) + if err != nil { + return nil, err + } + return q.ChunkQuerier(ctx, mint, maxt) +} + +// Blocks returns a slice of block readers for persisted blocks. +func (db *DBReadOnly) Blocks() ([]BlockReader, error) { + select { + case <-db.closed: + return nil, ErrClosed + default: + } + loadable, corrupted, err := openBlocks(db.logger, db.dir, nil, nil) + if err != nil { + return nil, err + } + + // Corrupted blocks that have been superseded by a loadable block can be safely ignored. + for _, block := range loadable { + for _, b := range block.Meta().Compaction.Parents { + delete(corrupted, b.ULID) + } + } + if len(corrupted) > 0 { + for _, b := range loadable { + if err := b.Close(); err != nil { + level.Warn(db.logger).Log("msg", "Closing block failed", "err", err, "block", b) + } + } + errs := tsdb_errors.NewMulti() + for ulid, err := range corrupted { + errs.Add(errors.Wrapf(err, "corrupted block %s", ulid.String())) + } + return nil, errs.Err() + } + + if len(loadable) == 0 { + return nil, nil + } + + sort.Slice(loadable, func(i, j int) bool { + return loadable[i].Meta().MinTime < loadable[j].Meta().MinTime + }) + + blockMetas := make([]BlockMeta, 0, len(loadable)) + for _, b := range loadable { + blockMetas = append(blockMetas, b.Meta()) + } + if overlaps := OverlappingBlocks(blockMetas); len(overlaps) > 0 { + level.Warn(db.logger).Log("msg", "Overlapping blocks found during opening", "detail", overlaps.String()) + } + + // Close all previously open readers and add the new ones to the cache. + for _, closer := range db.closers { + closer.Close() + } + + blockClosers := make([]io.Closer, len(loadable)) + blockReaders := make([]BlockReader, len(loadable)) + for i, b := range loadable { + blockClosers[i] = b + blockReaders[i] = b + } + db.closers = blockClosers + + return blockReaders, nil +} + +// Close all block readers. +func (db *DBReadOnly) Close() error { + select { + case <-db.closed: + return ErrClosed + default: + } + close(db.closed) + + return tsdb_errors.CloseAll(db.closers) +} + +// Open returns a new DB in the given directory. If options are empty, DefaultOptions will be used. +func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, stats *DBStats) (db *DB, err error) { + var rngs []int64 + opts, rngs = validateOpts(opts, nil) + + return open(dir, l, r, opts, rngs, stats) +} + +func validateOpts(opts *Options, rngs []int64) (*Options, []int64) { + if opts == nil { + opts = DefaultOptions() + } + if opts.StripeSize <= 0 { + opts.StripeSize = DefaultStripeSize + } + if opts.HeadChunksWriteBufferSize <= 0 { + opts.HeadChunksWriteBufferSize = chunks.DefaultWriteBufferSize + } + if opts.HeadChunksWriteQueueSize < 0 { + opts.HeadChunksWriteQueueSize = chunks.DefaultWriteQueueSize + } + if opts.MaxBlockChunkSegmentSize <= 0 { + opts.MaxBlockChunkSegmentSize = chunks.DefaultChunkSegmentSize + } + if opts.MinBlockDuration <= 0 { + opts.MinBlockDuration = DefaultBlockDuration + } + if opts.MinBlockDuration > opts.MaxBlockDuration { + opts.MaxBlockDuration = opts.MinBlockDuration + } + + if len(rngs) == 0 { + // Start with smallest block duration and create exponential buckets until the exceed the + // configured maximum block duration. + rngs = ExponentialBlockRanges(opts.MinBlockDuration, 10, 3) + } + return opts, rngs +} + +// open returns a new DB in the given directory. +// It initializes the lockfile, WAL, compactor, and Head (by replaying the WAL), and runs the database. +// It is not safe to open more than one DB in the same directory. +func open(dir string, l log.Logger, r prometheus.Registerer, opts *Options, rngs []int64, stats *DBStats) (_ *DB, returnedErr error) { + if err := os.MkdirAll(dir, 0o777); err != nil { + return nil, err + } + if l == nil { + l = log.NewNopLogger() + } + if stats == nil { + stats = NewDBStats() + } + + for i, v := range rngs { + if v > opts.MaxBlockDuration { + rngs = rngs[:i] + break + } + } + + // Fixup bad format written by Prometheus 2.1. + if err := repairBadIndexVersion(l, dir); err != nil { + return nil, errors.Wrap(err, "repair bad index version") + } + + walDir := filepath.Join(dir, "wal") + + // Migrate old WAL if one exists. + if err := MigrateWAL(l, walDir); err != nil { + return nil, errors.Wrap(err, "migrate WAL") + } + // Remove garbage, tmp blocks. + if err := removeBestEffortTmpDirs(l, dir); err != nil { + return nil, errors.Wrap(err, "remove tmp dirs") + } + + db := &DB{ + dir: dir, + logger: l, + opts: opts, + compactc: make(chan struct{}, 1), + donec: make(chan struct{}), + stopc: make(chan struct{}), + autoCompact: true, + chunkPool: chunkenc.NewPool(), + blocksToDelete: opts.BlocksToDelete, + } + defer func() { + // Close files if startup fails somewhere. + if returnedErr == nil { + return + } + + close(db.donec) // DB is never run if it was an error, so close this channel here. + + returnedErr = tsdb_errors.NewMulti( + returnedErr, + errors.Wrap(db.Close(), "close DB after failed startup"), + ).Err() + }() + + if db.blocksToDelete == nil { + db.blocksToDelete = DefaultBlocksToDelete(db) + } + + var err error + db.locker, err = tsdbutil.NewDirLocker(dir, "tsdb", db.logger, r) + if err != nil { + return nil, err + } + if !opts.NoLockfile { + if err := db.locker.Lock(); err != nil { + return nil, err + } + } + + ctx, cancel := context.WithCancel(context.Background()) + db.compactor, err = NewLeveledCompactorWithChunkSize(ctx, r, l, rngs, db.chunkPool, opts.MaxBlockChunkSegmentSize, nil) + if err != nil { + cancel() + return nil, errors.Wrap(err, "create leveled compactor") + } + db.compactCancel = cancel + + var wlog *wal.WAL + segmentSize := wal.DefaultSegmentSize + // Wal is enabled. + if opts.WALSegmentSize >= 0 { + // Wal is set to a custom size. + if opts.WALSegmentSize > 0 { + segmentSize = opts.WALSegmentSize + } + wlog, err = wal.NewSize(l, r, walDir, segmentSize, opts.WALCompression) + if err != nil { + return nil, err + } + } + + headOpts := DefaultHeadOptions() + headOpts.ChunkRange = rngs[0] + headOpts.ChunkDirRoot = dir + headOpts.ChunkPool = db.chunkPool + headOpts.ChunkWriteBufferSize = opts.HeadChunksWriteBufferSize + headOpts.ChunkWriteQueueSize = opts.HeadChunksWriteQueueSize + headOpts.StripeSize = opts.StripeSize + headOpts.SeriesCallback = opts.SeriesLifecycleCallback + headOpts.EnableExemplarStorage = opts.EnableExemplarStorage + headOpts.MaxExemplars.Store(opts.MaxExemplars) + headOpts.EnableMemorySnapshotOnShutdown = opts.EnableMemorySnapshotOnShutdown + if opts.IsolationDisabled { + // We only override this flag if isolation is disabled at DB level. We use the default otherwise. + headOpts.IsolationDisabled = opts.IsolationDisabled + } + db.head, err = NewHead(r, l, wlog, headOpts, stats.Head) + if err != nil { + return nil, err + } + + // Register metrics after assigning the head block. + db.metrics = newDBMetrics(db, r) + maxBytes := opts.MaxBytes + if maxBytes < 0 { + maxBytes = 0 + } + db.metrics.maxBytes.Set(float64(maxBytes)) + + if err := db.reload(); err != nil { + return nil, err + } + // Set the min valid time for the ingested samples + // to be no lower than the maxt of the last block. + blocks := db.Blocks() + minValidTime := int64(math.MinInt64) + if len(blocks) > 0 { + minValidTime = blocks[len(blocks)-1].Meta().MaxTime + } + + if initErr := db.head.Init(minValidTime); initErr != nil { + db.head.metrics.walCorruptionsTotal.Inc() + level.Warn(db.logger).Log("msg", "Encountered WAL read error, attempting repair", "err", initErr) + if err := wlog.Repair(initErr); err != nil { + return nil, errors.Wrap(err, "repair corrupted WAL") + } + } + + go db.run() + + return db, nil +} + +func removeBestEffortTmpDirs(l log.Logger, dir string) error { + files, err := ioutil.ReadDir(dir) + if err != nil { + return err + } + for _, fi := range files { + if isTmpBlockDir(fi) { + if err := os.RemoveAll(filepath.Join(dir, fi.Name())); err != nil { + level.Error(l).Log("msg", "failed to delete tmp block dir", "dir", filepath.Join(dir, fi.Name()), "err", err) + continue + } + level.Info(l).Log("msg", "Found and deleted tmp block dir", "dir", filepath.Join(dir, fi.Name())) + } + } + return nil +} + +// StartTime implements the Storage interface. +func (db *DB) StartTime() (int64, error) { + db.mtx.RLock() + defer db.mtx.RUnlock() + + if len(db.blocks) > 0 { + return db.blocks[0].Meta().MinTime, nil + } + return db.head.MinTime(), nil +} + +// Dir returns the directory of the database. +func (db *DB) Dir() string { + return db.dir +} + +func (db *DB) run() { + defer close(db.donec) + + backoff := time.Duration(0) + + for { + select { + case <-db.stopc: + return + case <-time.After(backoff): + } + + select { + case <-time.After(1 * time.Minute): + db.cmtx.Lock() + if err := db.reloadBlocks(); err != nil { + level.Error(db.logger).Log("msg", "reloadBlocks", "err", err) + } + db.cmtx.Unlock() + + select { + case db.compactc <- struct{}{}: + default: + } + case <-db.compactc: + db.metrics.compactionsTriggered.Inc() + + db.autoCompactMtx.Lock() + if db.autoCompact { + if err := db.Compact(); err != nil { + level.Error(db.logger).Log("msg", "compaction failed", "err", err) + backoff = exponential(backoff, 1*time.Second, 1*time.Minute) + } else { + backoff = 0 + } + } else { + db.metrics.compactionsSkipped.Inc() + } + db.autoCompactMtx.Unlock() + case <-db.stopc: + return + } + } +} + +// Appender opens a new appender against the database. +func (db *DB) Appender(ctx context.Context) storage.Appender { + return dbAppender{db: db, Appender: db.head.Appender(ctx)} +} + +func (db *DB) ApplyConfig(conf *config.Config) error { + return db.head.ApplyConfig(conf) +} + +// dbAppender wraps the DB's head appender and triggers compactions on commit +// if necessary. +type dbAppender struct { + storage.Appender + db *DB +} + +var _ storage.GetRef = dbAppender{} + +func (a dbAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { + if g, ok := a.Appender.(storage.GetRef); ok { + return g.GetRef(lset) + } + return 0, nil +} + +func (a dbAppender) Commit() error { + err := a.Appender.Commit() + + // We could just run this check every few minutes practically. But for benchmarks + // and high frequency use cases this is the safer way. + if a.db.head.compactable() { + select { + case a.db.compactc <- struct{}{}: + default: + } + } + return err +} + +// Compact data if possible. After successful compaction blocks are reloaded +// which will also delete the blocks that fall out of the retention window. +// Old blocks are only deleted on reloadBlocks based on the new block's parent information. +// See DB.reloadBlocks documentation for further information. +func (db *DB) Compact() (returnErr error) { + db.cmtx.Lock() + defer db.cmtx.Unlock() + defer func() { + if returnErr != nil { + db.metrics.compactionsFailed.Inc() + } + }() + + lastBlockMaxt := int64(math.MinInt64) + defer func() { + returnErr = tsdb_errors.NewMulti( + returnErr, + errors.Wrap(db.head.truncateWAL(lastBlockMaxt), "WAL truncation in Compact defer"), + ).Err() + }() + + start := time.Now() + // Check whether we have pending head blocks that are ready to be persisted. + // They have the highest priority. + for { + select { + case <-db.stopc: + return nil + default: + } + if !db.head.compactable() { + break + } + mint := db.head.MinTime() + maxt := rangeForTimestamp(mint, db.head.chunkRange.Load()) + + // Wrap head into a range that bounds all reads to it. + // We remove 1 millisecond from maxt because block + // intervals are half-open: [b.MinTime, b.MaxTime). But + // chunk intervals are closed: [c.MinTime, c.MaxTime]; + // so in order to make sure that overlaps are evaluated + // consistently, we explicitly remove the last value + // from the block interval here. + if err := db.compactHead(NewRangeHead(db.head, mint, maxt-1)); err != nil { + return errors.Wrap(err, "compact head") + } + // Consider only successful compactions for WAL truncation. + lastBlockMaxt = maxt + } + + // Clear some disk space before compacting blocks, especially important + // when Head compaction happened over a long time range. + if err := db.head.truncateWAL(lastBlockMaxt); err != nil { + return errors.Wrap(err, "WAL truncation in Compact") + } + + compactionDuration := time.Since(start) + if compactionDuration.Milliseconds() > db.head.chunkRange.Load() { + level.Warn(db.logger).Log( + "msg", "Head compaction took longer than the block time range, compactions are falling behind and won't be able to catch up", + "duration", compactionDuration.String(), + "block_range", db.head.chunkRange.Load(), + ) + } + return db.compactBlocks() +} + +// CompactHead compacts the given RangeHead. +func (db *DB) CompactHead(head *RangeHead) error { + db.cmtx.Lock() + defer db.cmtx.Unlock() + + if err := db.compactHead(head); err != nil { + return errors.Wrap(err, "compact head") + } + + if err := db.head.truncateWAL(head.BlockMaxTime()); err != nil { + return errors.Wrap(err, "WAL truncation") + } + return nil +} + +// compactHead compacts the given RangeHead. +// The compaction mutex should be held before calling this method. +func (db *DB) compactHead(head *RangeHead) error { + uid, err := db.compactor.Write(db.dir, head, head.MinTime(), head.BlockMaxTime(), nil) + if err != nil { + return errors.Wrap(err, "persist head block") + } + + if err := db.reloadBlocks(); err != nil { + if errRemoveAll := os.RemoveAll(filepath.Join(db.dir, uid.String())); errRemoveAll != nil { + return tsdb_errors.NewMulti( + errors.Wrap(err, "reloadBlocks blocks"), + errors.Wrapf(errRemoveAll, "delete persisted head block after failed db reloadBlocks:%s", uid), + ).Err() + } + return errors.Wrap(err, "reloadBlocks blocks") + } + if err = db.head.truncateMemory(head.BlockMaxTime()); err != nil { + return errors.Wrap(err, "head memory truncate") + } + return nil +} + +// compactBlocks compacts all the eligible on-disk blocks. +// The compaction mutex should be held before calling this method. +func (db *DB) compactBlocks() (err error) { + // Check for compactions of multiple blocks. + for { + plan, err := db.compactor.Plan(db.dir) + if err != nil { + return errors.Wrap(err, "plan compaction") + } + if len(plan) == 0 { + break + } + + select { + case <-db.stopc: + return nil + default: + } + + uid, err := db.compactor.Compact(db.dir, plan, db.blocks) + if err != nil { + return errors.Wrapf(err, "compact %s", plan) + } + + if err := db.reloadBlocks(); err != nil { + if err := os.RemoveAll(filepath.Join(db.dir, uid.String())); err != nil { + return errors.Wrapf(err, "delete compacted block after failed db reloadBlocks:%s", uid) + } + return errors.Wrap(err, "reloadBlocks blocks") + } + } + + return nil +} + +// getBlock iterates a given block range to find a block by a given id. +// If found it returns the block itself and a boolean to indicate that it was found. +func getBlock(allBlocks []*Block, id ulid.ULID) (*Block, bool) { + for _, b := range allBlocks { + if b.Meta().ULID == id { + return b, true + } + } + return nil, false +} + +// reload reloads blocks and truncates the head and its WAL. +func (db *DB) reload() error { + if err := db.reloadBlocks(); err != nil { + return errors.Wrap(err, "reloadBlocks") + } + if len(db.blocks) == 0 { + return nil + } + if err := db.head.Truncate(db.blocks[len(db.blocks)-1].MaxTime()); err != nil { + return errors.Wrap(err, "head truncate") + } + return nil +} + +// reloadBlocks reloads blocks without touching head. +// Blocks that are obsolete due to replacement or retention will be deleted. +func (db *DB) reloadBlocks() (err error) { + defer func() { + if err != nil { + db.metrics.reloadsFailed.Inc() + } + db.metrics.reloads.Inc() + }() + + // Now that we reload TSDB every minute, there is high chance for race condition with a reload + // triggered by CleanTombstones(). We need to lock the reload to avoid the situation where + // a normal reload and CleanTombstones try to delete the same block. + db.mtx.Lock() + defer db.mtx.Unlock() + + loadable, corrupted, err := openBlocks(db.logger, db.dir, db.blocks, db.chunkPool) + if err != nil { + return err + } + + deletableULIDs := db.blocksToDelete(loadable) + deletable := make(map[ulid.ULID]*Block, len(deletableULIDs)) + + // Mark all parents of loaded blocks as deletable (no matter if they exists). This makes it resilient against the process + // crashing towards the end of a compaction but before deletions. By doing that, we can pick up the deletion where it left off during a crash. + for _, block := range loadable { + if _, ok := deletableULIDs[block.meta.ULID]; ok { + deletable[block.meta.ULID] = block + } + for _, b := range block.Meta().Compaction.Parents { + if _, ok := corrupted[b.ULID]; ok { + delete(corrupted, b.ULID) + level.Warn(db.logger).Log("msg", "Found corrupted block, but replaced by compacted one so it's safe to delete. This should not happen with atomic deletes.", "block", b.ULID) + } + deletable[b.ULID] = nil + } + } + + if len(corrupted) > 0 { + // Corrupted but no child loaded for it. + // Close all new blocks to release the lock for windows. + for _, block := range loadable { + if _, open := getBlock(db.blocks, block.Meta().ULID); !open { + block.Close() + } + } + errs := tsdb_errors.NewMulti() + for ulid, err := range corrupted { + errs.Add(errors.Wrapf(err, "corrupted block %s", ulid.String())) + } + return errs.Err() + } + + var ( + toLoad []*Block + blocksSize int64 + ) + // All deletable blocks should be unloaded. + // NOTE: We need to loop through loadable one more time as there might be loadable ready to be removed (replaced by compacted block). + for _, block := range loadable { + if _, ok := deletable[block.Meta().ULID]; ok { + deletable[block.Meta().ULID] = block + continue + } + + toLoad = append(toLoad, block) + blocksSize += block.Size() + } + db.metrics.blocksBytes.Set(float64(blocksSize)) + + sort.Slice(toLoad, func(i, j int) bool { + return toLoad[i].Meta().MinTime < toLoad[j].Meta().MinTime + }) + if !db.opts.AllowOverlappingBlocks { + if err := validateBlockSequence(toLoad); err != nil { + return errors.Wrap(err, "invalid block sequence") + } + } + + // Swap new blocks first for subsequently created readers to be seen. + oldBlocks := db.blocks + db.blocks = toLoad + + blockMetas := make([]BlockMeta, 0, len(toLoad)) + for _, b := range toLoad { + blockMetas = append(blockMetas, b.Meta()) + } + if overlaps := OverlappingBlocks(blockMetas); len(overlaps) > 0 { + level.Warn(db.logger).Log("msg", "Overlapping blocks found during reloadBlocks", "detail", overlaps.String()) + } + + // Append blocks to old, deletable blocks, so we can close them. + for _, b := range oldBlocks { + if _, ok := deletable[b.Meta().ULID]; ok { + deletable[b.Meta().ULID] = b + } + } + if err := db.deleteBlocks(deletable); err != nil { + return errors.Wrapf(err, "delete %v blocks", len(deletable)) + } + return nil +} + +func openBlocks(l log.Logger, dir string, loaded []*Block, chunkPool chunkenc.Pool) (blocks []*Block, corrupted map[ulid.ULID]error, err error) { + bDirs, err := blockDirs(dir) + if err != nil { + return nil, nil, errors.Wrap(err, "find blocks") + } + + corrupted = make(map[ulid.ULID]error) + for _, bDir := range bDirs { + meta, _, err := readMetaFile(bDir) + if err != nil { + level.Error(l).Log("msg", "Failed to read meta.json for a block during reloadBlocks. Skipping", "dir", bDir, "err", err) + continue + } + + // See if we already have the block in memory or open it otherwise. + block, open := getBlock(loaded, meta.ULID) + if !open { + block, err = OpenBlock(l, bDir, chunkPool) + if err != nil { + corrupted[meta.ULID] = err + continue + } + } + blocks = append(blocks, block) + } + return blocks, corrupted, nil +} + +// DefaultBlocksToDelete returns a filter which decides time based and size based +// retention from the options of the db. +func DefaultBlocksToDelete(db *DB) BlocksToDeleteFunc { + return func(blocks []*Block) map[ulid.ULID]struct{} { + return deletableBlocks(db, blocks) + } +} + +// deletableBlocks returns all currently loaded blocks past retention policy or already compacted into a new block. +func deletableBlocks(db *DB, blocks []*Block) map[ulid.ULID]struct{} { + deletable := make(map[ulid.ULID]struct{}) + + // Sort the blocks by time - newest to oldest (largest to smallest timestamp). + // This ensures that the retentions will remove the oldest blocks. + sort.Slice(blocks, func(i, j int) bool { + return blocks[i].Meta().MaxTime > blocks[j].Meta().MaxTime + }) + + for _, block := range blocks { + if block.Meta().Compaction.Deletable { + deletable[block.Meta().ULID] = struct{}{} + } + } + + for ulid := range BeyondTimeRetention(db, blocks) { + deletable[ulid] = struct{}{} + } + + for ulid := range BeyondSizeRetention(db, blocks) { + deletable[ulid] = struct{}{} + } + + return deletable +} + +// BeyondTimeRetention returns those blocks which are beyond the time retention +// set in the db options. +func BeyondTimeRetention(db *DB, blocks []*Block) (deletable map[ulid.ULID]struct{}) { + // Time retention is disabled or no blocks to work with. + if len(blocks) == 0 || db.opts.RetentionDuration == 0 { + return + } + + deletable = make(map[ulid.ULID]struct{}) + for i, block := range blocks { + // The difference between the first block and this block is larger than + // the retention period so any blocks after that are added as deletable. + if i > 0 && blocks[0].Meta().MaxTime-block.Meta().MaxTime > db.opts.RetentionDuration { + for _, b := range blocks[i:] { + deletable[b.meta.ULID] = struct{}{} + } + db.metrics.timeRetentionCount.Inc() + break + } + } + return deletable +} + +// BeyondSizeRetention returns those blocks which are beyond the size retention +// set in the db options. +func BeyondSizeRetention(db *DB, blocks []*Block) (deletable map[ulid.ULID]struct{}) { + // Size retention is disabled or no blocks to work with. + if len(blocks) == 0 || db.opts.MaxBytes <= 0 { + return + } + + deletable = make(map[ulid.ULID]struct{}) + + // Initializing size counter with WAL size and Head chunks + // written to disk, as that is part of the retention strategy. + blocksSize := db.Head().Size() + for i, block := range blocks { + blocksSize += block.Size() + if blocksSize > int64(db.opts.MaxBytes) { + // Add this and all following blocks for deletion. + for _, b := range blocks[i:] { + deletable[b.meta.ULID] = struct{}{} + } + db.metrics.sizeRetentionCount.Inc() + break + } + } + return deletable +} + +// deleteBlocks closes the block if loaded and deletes blocks from the disk if exists. +// When the map contains a non nil block object it means it is loaded in memory +// so needs to be closed first as it might need to wait for pending readers to complete. +func (db *DB) deleteBlocks(blocks map[ulid.ULID]*Block) error { + for ulid, block := range blocks { + if block != nil { + if err := block.Close(); err != nil { + level.Warn(db.logger).Log("msg", "Closing block failed", "err", err, "block", ulid) + } + } + + toDelete := filepath.Join(db.dir, ulid.String()) + if _, err := os.Stat(toDelete); os.IsNotExist(err) { + // Noop. + continue + } else if err != nil { + return errors.Wrapf(err, "stat dir %v", toDelete) + } + + // Replace atomically to avoid partial block when process would crash during deletion. + tmpToDelete := filepath.Join(db.dir, fmt.Sprintf("%s%s", ulid, tmpForDeletionBlockDirSuffix)) + if err := fileutil.Replace(toDelete, tmpToDelete); err != nil { + return errors.Wrapf(err, "replace of obsolete block for deletion %s", ulid) + } + if err := os.RemoveAll(tmpToDelete); err != nil { + return errors.Wrapf(err, "delete obsolete block %s", ulid) + } + level.Info(db.logger).Log("msg", "Deleting obsolete block", "block", ulid) + } + + return nil +} + +// validateBlockSequence returns error if given block meta files indicate that some blocks overlaps within sequence. +func validateBlockSequence(bs []*Block) error { + if len(bs) <= 1 { + return nil + } + + var metas []BlockMeta + for _, b := range bs { + metas = append(metas, b.meta) + } + + overlaps := OverlappingBlocks(metas) + if len(overlaps) > 0 { + return errors.Errorf("block time ranges overlap: %s", overlaps) + } + + return nil +} + +// TimeRange specifies minTime and maxTime range. +type TimeRange struct { + Min, Max int64 +} + +// Overlaps contains overlapping blocks aggregated by overlapping range. +type Overlaps map[TimeRange][]BlockMeta + +// String returns human readable string form of overlapped blocks. +func (o Overlaps) String() string { + var res []string + for r, overlaps := range o { + var groups []string + for _, m := range overlaps { + groups = append(groups, fmt.Sprintf( + "", + m.ULID.String(), + m.MinTime, + m.MaxTime, + (time.Duration((m.MaxTime-m.MinTime)/1000)*time.Second).String(), + )) + } + res = append(res, fmt.Sprintf( + "[mint: %d, maxt: %d, range: %s, blocks: %d]: %s", + r.Min, r.Max, + (time.Duration((r.Max-r.Min)/1000)*time.Second).String(), + len(overlaps), + strings.Join(groups, ", ")), + ) + } + return strings.Join(res, "\n") +} + +// OverlappingBlocks returns all overlapping blocks from given meta files. +func OverlappingBlocks(bm []BlockMeta) Overlaps { + if len(bm) <= 1 { + return nil + } + var ( + overlaps [][]BlockMeta + + // pending contains not ended blocks in regards to "current" timestamp. + pending = []BlockMeta{bm[0]} + // continuousPending helps to aggregate same overlaps to single group. + continuousPending = true + ) + + // We have here blocks sorted by minTime. We iterate over each block and treat its minTime as our "current" timestamp. + // We check if any of the pending block finished (blocks that we have seen before, but their maxTime was still ahead current + // timestamp). If not, it means they overlap with our current block. In the same time current block is assumed pending. + for _, b := range bm[1:] { + var newPending []BlockMeta + + for _, p := range pending { + // "b.MinTime" is our current time. + if b.MinTime >= p.MaxTime { + continuousPending = false + continue + } + + // "p" overlaps with "b" and "p" is still pending. + newPending = append(newPending, p) + } + + // Our block "b" is now pending. + pending = append(newPending, b) + if len(newPending) == 0 { + // No overlaps. + continue + } + + if continuousPending && len(overlaps) > 0 { + overlaps[len(overlaps)-1] = append(overlaps[len(overlaps)-1], b) + continue + } + overlaps = append(overlaps, append(newPending, b)) + // Start new pendings. + continuousPending = true + } + + // Fetch the critical overlapped time range foreach overlap groups. + overlapGroups := Overlaps{} + for _, overlap := range overlaps { + + minRange := TimeRange{Min: 0, Max: math.MaxInt64} + for _, b := range overlap { + if minRange.Max > b.MaxTime { + minRange.Max = b.MaxTime + } + + if minRange.Min < b.MinTime { + minRange.Min = b.MinTime + } + } + overlapGroups[minRange] = overlap + } + + return overlapGroups +} + +func (db *DB) String() string { + return "HEAD" +} + +// Blocks returns the databases persisted blocks. +func (db *DB) Blocks() []*Block { + db.mtx.RLock() + defer db.mtx.RUnlock() + + return db.blocks +} + +// Head returns the databases's head. +func (db *DB) Head() *Head { + return db.head +} + +// Close the partition. +func (db *DB) Close() error { + close(db.stopc) + if db.compactCancel != nil { + db.compactCancel() + } + <-db.donec + + db.mtx.Lock() + defer db.mtx.Unlock() + + var g errgroup.Group + + // blocks also contains all head blocks. + for _, pb := range db.blocks { + g.Go(pb.Close) + } + + errs := tsdb_errors.NewMulti(g.Wait(), db.locker.Release()) + if db.head != nil { + errs.Add(db.head.Close()) + } + return errs.Err() +} + +// DisableCompactions disables auto compactions. +func (db *DB) DisableCompactions() { + db.autoCompactMtx.Lock() + defer db.autoCompactMtx.Unlock() + + db.autoCompact = false + level.Info(db.logger).Log("msg", "Compactions disabled") +} + +// EnableCompactions enables auto compactions. +func (db *DB) EnableCompactions() { + db.autoCompactMtx.Lock() + defer db.autoCompactMtx.Unlock() + + db.autoCompact = true + level.Info(db.logger).Log("msg", "Compactions enabled") +} + +// Snapshot writes the current data to the directory. If withHead is set to true it +// will create a new block containing all data that's currently in the memory buffer/WAL. +func (db *DB) Snapshot(dir string, withHead bool) error { + if dir == db.dir { + return errors.Errorf("cannot snapshot into base directory") + } + if _, err := ulid.ParseStrict(dir); err == nil { + return errors.Errorf("dir must not be a valid ULID") + } + + db.cmtx.Lock() + defer db.cmtx.Unlock() + + db.mtx.RLock() + defer db.mtx.RUnlock() + + for _, b := range db.blocks { + level.Info(db.logger).Log("msg", "Snapshotting block", "block", b) + + if err := b.Snapshot(dir); err != nil { + return errors.Wrapf(err, "error snapshotting block: %s", b.Dir()) + } + } + if !withHead { + return nil + } + + mint := db.head.MinTime() + maxt := db.head.MaxTime() + head := NewRangeHead(db.head, mint, maxt) + // Add +1 millisecond to block maxt because block intervals are half-open: [b.MinTime, b.MaxTime). + // Because of this block intervals are always +1 than the total samples it includes. + if _, err := db.compactor.Write(dir, head, mint, maxt+1, nil); err != nil { + return errors.Wrap(err, "snapshot head block") + } + return nil +} + +// Querier returns a new querier over the data partition for the given time range. +func (db *DB) Querier(_ context.Context, mint, maxt int64) (storage.Querier, error) { + var blocks []BlockReader + + db.mtx.RLock() + defer db.mtx.RUnlock() + + for _, b := range db.blocks { + if b.OverlapsClosedInterval(mint, maxt) { + blocks = append(blocks, b) + } + } + var headQuerier storage.Querier + if maxt >= db.head.MinTime() { + rh := NewRangeHead(db.head, mint, maxt) + var err error + headQuerier, err = NewBlockQuerier(rh, mint, maxt) + if err != nil { + return nil, errors.Wrapf(err, "open querier for head %s", rh) + } + + // Getting the querier above registers itself in the queue that the truncation waits on. + // So if the querier is currently not colliding with any truncation, we can continue to use it and still + // won't run into a race later since any truncation that comes after will wait on this querier if it overlaps. + shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(mint, maxt) + if shouldClose { + if err := headQuerier.Close(); err != nil { + return nil, errors.Wrapf(err, "closing head querier %s", rh) + } + headQuerier = nil + } + if getNew { + rh := NewRangeHead(db.head, newMint, maxt) + headQuerier, err = NewBlockQuerier(rh, newMint, maxt) + if err != nil { + return nil, errors.Wrapf(err, "open querier for head while getting new querier %s", rh) + } + } + } + + blockQueriers := make([]storage.Querier, 0, len(blocks)) + for _, b := range blocks { + q, err := NewBlockQuerier(b, mint, maxt) + if err == nil { + blockQueriers = append(blockQueriers, q) + continue + } + // If we fail, all previously opened queriers must be closed. + for _, q := range blockQueriers { + // TODO(bwplotka): Handle error. + _ = q.Close() + } + return nil, errors.Wrapf(err, "open querier for block %s", b) + } + if headQuerier != nil { + blockQueriers = append(blockQueriers, headQuerier) + } + return storage.NewMergeQuerier(blockQueriers, nil, storage.ChainedSeriesMerge), nil +} + +// ChunkQuerier returns a new chunk querier over the data partition for the given time range. +func (db *DB) ChunkQuerier(_ context.Context, mint, maxt int64) (storage.ChunkQuerier, error) { + var blocks []BlockReader + + db.mtx.RLock() + defer db.mtx.RUnlock() + + for _, b := range db.blocks { + if b.OverlapsClosedInterval(mint, maxt) { + blocks = append(blocks, b) + } + } + var headQuerier storage.ChunkQuerier + if maxt >= db.head.MinTime() { + rh := NewRangeHead(db.head, mint, maxt) + var err error + headQuerier, err = NewBlockChunkQuerier(rh, mint, maxt) + if err != nil { + return nil, errors.Wrapf(err, "open querier for head %s", rh) + } + + // Getting the querier above registers itself in the queue that the truncation waits on. + // So if the querier is currently not colliding with any truncation, we can continue to use it and still + // won't run into a race later since any truncation that comes after will wait on this querier if it overlaps. + shouldClose, getNew, newMint := db.head.IsQuerierCollidingWithTruncation(mint, maxt) + if shouldClose { + if err := headQuerier.Close(); err != nil { + return nil, errors.Wrapf(err, "closing head querier %s", rh) + } + headQuerier = nil + } + if getNew { + rh := NewRangeHead(db.head, newMint, maxt) + headQuerier, err = NewBlockChunkQuerier(rh, newMint, maxt) + if err != nil { + return nil, errors.Wrapf(err, "open querier for head while getting new querier %s", rh) + } + } + } + + blockQueriers := make([]storage.ChunkQuerier, 0, len(blocks)) + for _, b := range blocks { + q, err := NewBlockChunkQuerier(b, mint, maxt) + if err == nil { + blockQueriers = append(blockQueriers, q) + continue + } + // If we fail, all previously opened queriers must be closed. + for _, q := range blockQueriers { + // TODO(bwplotka): Handle error. + _ = q.Close() + } + return nil, errors.Wrapf(err, "open querier for block %s", b) + } + if headQuerier != nil { + blockQueriers = append(blockQueriers, headQuerier) + } + + return storage.NewMergeChunkQuerier(blockQueriers, nil, storage.NewCompactingChunkSeriesMerger(storage.ChainedSeriesMerge)), nil +} + +func (db *DB) ExemplarQuerier(ctx context.Context) (storage.ExemplarQuerier, error) { + return db.head.exemplars.ExemplarQuerier(ctx) +} + +func rangeForTimestamp(t, width int64) (maxt int64) { + return (t/width)*width + width +} + +// Delete implements deletion of metrics. It only has atomicity guarantees on a per-block basis. +func (db *DB) Delete(mint, maxt int64, ms ...*labels.Matcher) error { + db.cmtx.Lock() + defer db.cmtx.Unlock() + + var g errgroup.Group + + db.mtx.RLock() + defer db.mtx.RUnlock() + + for _, b := range db.blocks { + if b.OverlapsClosedInterval(mint, maxt) { + g.Go(func(b *Block) func() error { + return func() error { return b.Delete(mint, maxt, ms...) } + }(b)) + } + } + if db.head.OverlapsClosedInterval(mint, maxt) { + g.Go(func() error { + return db.head.Delete(mint, maxt, ms...) + }) + } + + return g.Wait() +} + +// CleanTombstones re-writes any blocks with tombstones. +func (db *DB) CleanTombstones() (err error) { + db.cmtx.Lock() + defer db.cmtx.Unlock() + + start := time.Now() + defer db.metrics.tombCleanTimer.Observe(time.Since(start).Seconds()) + + cleanUpCompleted := false + // Repeat cleanup until there is no tombstones left. + for !cleanUpCompleted { + cleanUpCompleted = true + + for _, pb := range db.Blocks() { + uid, safeToDelete, cleanErr := pb.CleanTombstones(db.Dir(), db.compactor) + if cleanErr != nil { + return errors.Wrapf(cleanErr, "clean tombstones: %s", pb.Dir()) + } + if !safeToDelete { + // There was nothing to clean. + continue + } + + // In case tombstones of the old block covers the whole block, + // then there would be no resultant block to tell the parent. + // The lock protects against race conditions when deleting blocks + // during an already running reload. + db.mtx.Lock() + pb.meta.Compaction.Deletable = safeToDelete + db.mtx.Unlock() + cleanUpCompleted = false + if err = db.reloadBlocks(); err == nil { // Will try to delete old block. + // Successful reload will change the existing blocks. + // We need to loop over the new set of blocks. + break + } + + // Delete new block if it was created. + if uid != nil && *uid != (ulid.ULID{}) { + dir := filepath.Join(db.Dir(), uid.String()) + if err := os.RemoveAll(dir); err != nil { + level.Error(db.logger).Log("msg", "failed to delete block after failed `CleanTombstones`", "dir", dir, "err", err) + } + } + return errors.Wrap(err, "reload blocks") + } + } + return nil +} + +func isBlockDir(fi os.FileInfo) bool { + if !fi.IsDir() { + return false + } + _, err := ulid.ParseStrict(fi.Name()) + return err == nil +} + +// isTmpBlockDir returns dir that consists of block dir ULID and tmp extension. +func isTmpBlockDir(fi os.FileInfo) bool { + if !fi.IsDir() { + return false + } + + fn := fi.Name() + ext := filepath.Ext(fn) + if ext == tmpForDeletionBlockDirSuffix || ext == tmpForCreationBlockDirSuffix || ext == tmpLegacy { + if _, err := ulid.ParseStrict(fn[:len(fn)-len(ext)]); err == nil { + return true + } + } + return false +} + +func blockDirs(dir string) ([]string, error) { + files, err := ioutil.ReadDir(dir) + if err != nil { + return nil, err + } + var dirs []string + + for _, fi := range files { + if isBlockDir(fi) { + dirs = append(dirs, filepath.Join(dir, fi.Name())) + } + } + return dirs, nil +} + +func sequenceFiles(dir string) ([]string, error) { + files, err := ioutil.ReadDir(dir) + if err != nil { + return nil, err + } + var res []string + + for _, fi := range files { + if _, err := strconv.ParseUint(fi.Name(), 10, 64); err != nil { + continue + } + res = append(res, filepath.Join(dir, fi.Name())) + } + return res, nil +} + +func nextSequenceFile(dir string) (string, int, error) { + files, err := ioutil.ReadDir(dir) + if err != nil { + return "", 0, err + } + + i := uint64(0) + for _, f := range files { + j, err := strconv.ParseUint(f.Name(), 10, 64) + if err != nil { + continue + } + i = j + } + return filepath.Join(dir, fmt.Sprintf("%0.6d", i+1)), int(i + 1), nil +} + +func exponential(d, min, max time.Duration) time.Duration { + d *= 2 + if d < min { + d = min + } + if d > max { + d = max + } + return d +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/exemplar.go b/vendor/github.com/prometheus/prometheus/tsdb/exemplar.go new file mode 100644 index 00000000000..3718d9591d7 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/exemplar.go @@ -0,0 +1,431 @@ +// Copyright 2020 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "sort" + "sync" + "unicode/utf8" + + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" +) + +const ( + // Indicates that there is no index entry for an exmplar. + noExemplar = -1 + // Estimated number of exemplars per series, for sizing the index. + estimatedExemplarsPerSeries = 16 +) + +type CircularExemplarStorage struct { + lock sync.RWMutex + exemplars []*circularBufferEntry + nextIndex int + metrics *ExemplarMetrics + + // Map of series labels as a string to index entry, which points to the first + // and last exemplar for the series in the exemplars circular buffer. + index map[string]*indexEntry +} + +type indexEntry struct { + oldest int + newest int + seriesLabels labels.Labels +} + +type circularBufferEntry struct { + exemplar exemplar.Exemplar + next int + ref *indexEntry +} + +type ExemplarMetrics struct { + exemplarsAppended prometheus.Counter + exemplarsInStorage prometheus.Gauge + seriesWithExemplarsInStorage prometheus.Gauge + lastExemplarsTs prometheus.Gauge + maxExemplars prometheus.Gauge + outOfOrderExemplars prometheus.Counter +} + +func NewExemplarMetrics(reg prometheus.Registerer) *ExemplarMetrics { + m := ExemplarMetrics{ + exemplarsAppended: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_exemplar_exemplars_appended_total", + Help: "Total number of appended exemplars.", + }), + exemplarsInStorage: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_exemplar_exemplars_in_storage", + Help: "Number of exemplars currently in circular storage.", + }), + seriesWithExemplarsInStorage: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_exemplar_series_with_exemplars_in_storage", + Help: "Number of series with exemplars currently in circular storage.", + }), + lastExemplarsTs: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_exemplar_last_exemplars_timestamp_seconds", + Help: "The timestamp of the oldest exemplar stored in circular storage. Useful to check for what time" + + "range the current exemplar buffer limit allows. This usually means the last timestamp" + + "for all exemplars for a typical setup. This is not true though if one of the series timestamp is in future compared to rest series.", + }), + outOfOrderExemplars: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_exemplar_out_of_order_exemplars_total", + Help: "Total number of out of order exemplar ingestion failed attempts.", + }), + maxExemplars: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_exemplar_max_exemplars", + Help: "Total number of exemplars the exemplar storage can store, resizeable.", + }), + } + + if reg != nil { + reg.MustRegister( + m.exemplarsAppended, + m.exemplarsInStorage, + m.seriesWithExemplarsInStorage, + m.lastExemplarsTs, + m.outOfOrderExemplars, + m.maxExemplars, + ) + } + + return &m +} + +// NewCircularExemplarStorage creates an circular in memory exemplar storage. +// If we assume the average case 95 bytes per exemplar we can fit 5651272 exemplars in +// 1GB of extra memory, accounting for the fact that this is heap allocated space. +// If len <= 0, then the exemplar storage is essentially a noop storage but can later be +// resized to store exemplars. +func NewCircularExemplarStorage(len int64, m *ExemplarMetrics) (ExemplarStorage, error) { + if len < 0 { + len = 0 + } + c := &CircularExemplarStorage{ + exemplars: make([]*circularBufferEntry, len), + index: make(map[string]*indexEntry, len/estimatedExemplarsPerSeries), + metrics: m, + } + + c.metrics.maxExemplars.Set(float64(len)) + + return c, nil +} + +func (ce *CircularExemplarStorage) ApplyConfig(cfg *config.Config) error { + ce.Resize(cfg.StorageConfig.ExemplarsConfig.MaxExemplars) + return nil +} + +func (ce *CircularExemplarStorage) Appender() *CircularExemplarStorage { + return ce +} + +func (ce *CircularExemplarStorage) ExemplarQuerier(_ context.Context) (storage.ExemplarQuerier, error) { + return ce, nil +} + +func (ce *CircularExemplarStorage) Querier(_ context.Context) (storage.ExemplarQuerier, error) { + return ce, nil +} + +// Select returns exemplars for a given set of label matchers. +func (ce *CircularExemplarStorage) Select(start, end int64, matchers ...[]*labels.Matcher) ([]exemplar.QueryResult, error) { + ret := make([]exemplar.QueryResult, 0) + + if len(ce.exemplars) <= 0 { + return ret, nil + } + + ce.lock.RLock() + defer ce.lock.RUnlock() + + // Loop through each index entry, which will point us to first/last exemplar for each series. + for _, idx := range ce.index { + var se exemplar.QueryResult + e := ce.exemplars[idx.oldest] + if e.exemplar.Ts > end || ce.exemplars[idx.newest].exemplar.Ts < start { + continue + } + if !matchesSomeMatcherSet(idx.seriesLabels, matchers) { + continue + } + se.SeriesLabels = idx.seriesLabels + + // Loop through all exemplars in the circular buffer for the current series. + for e.exemplar.Ts <= end { + if e.exemplar.Ts >= start { + se.Exemplars = append(se.Exemplars, e.exemplar) + } + if e.next == noExemplar { + break + } + e = ce.exemplars[e.next] + } + if len(se.Exemplars) > 0 { + ret = append(ret, se) + } + } + + sort.Slice(ret, func(i, j int) bool { + return labels.Compare(ret[i].SeriesLabels, ret[j].SeriesLabels) < 0 + }) + + return ret, nil +} + +func matchesSomeMatcherSet(lbls labels.Labels, matchers [][]*labels.Matcher) bool { +Outer: + for _, ms := range matchers { + for _, m := range ms { + if !m.Matches(lbls.Get(m.Name)) { + continue Outer + } + } + return true + } + return false +} + +func (ce *CircularExemplarStorage) ValidateExemplar(l labels.Labels, e exemplar.Exemplar) error { + var buf [1024]byte + seriesLabels := l.Bytes(buf[:]) + + // TODO(bwplotka): This lock can lock all scrapers, there might high contention on this on scale. + // Optimize by moving the lock to be per series (& benchmark it). + ce.lock.RLock() + defer ce.lock.RUnlock() + return ce.validateExemplar(seriesLabels, e, false) +} + +// Not thread safe. The append parameters tells us whether this is an external validation, or internal +// as a result of an AddExemplar call, in which case we should update any relevant metrics. +func (ce *CircularExemplarStorage) validateExemplar(key []byte, e exemplar.Exemplar, append bool) error { + if len(ce.exemplars) <= 0 { + return storage.ErrExemplarsDisabled + } + + // Exemplar label length does not include chars involved in text rendering such as quotes + // equals sign, or commas. See definition of const ExemplarMaxLabelLength. + labelSetLen := 0 + for _, l := range e.Labels { + labelSetLen += utf8.RuneCountInString(l.Name) + labelSetLen += utf8.RuneCountInString(l.Value) + + if labelSetLen > exemplar.ExemplarMaxLabelSetLength { + return storage.ErrExemplarLabelLength + } + } + + idx, ok := ce.index[string(key)] + if !ok { + return nil + } + + // Check for duplicate vs last stored exemplar for this series. + // NB these are expected, and appending them is a no-op. + if ce.exemplars[idx.newest].exemplar.Equals(e) { + return storage.ErrDuplicateExemplar + } + + if e.Ts <= ce.exemplars[idx.newest].exemplar.Ts { + if append { + ce.metrics.outOfOrderExemplars.Inc() + } + return storage.ErrOutOfOrderExemplar + } + return nil +} + +// Resize changes the size of exemplar buffer by allocating a new buffer and migrating data to it. +// Exemplars are kept when possible. Shrinking will discard oldest data (in order of ingest) as needed. +func (ce *CircularExemplarStorage) Resize(l int64) int { + // Accept negative values as just 0 size. + if l <= 0 { + l = 0 + } + + if l == int64(len(ce.exemplars)) { + return 0 + } + + ce.lock.Lock() + defer ce.lock.Unlock() + + oldBuffer := ce.exemplars + oldNextIndex := int64(ce.nextIndex) + + ce.exemplars = make([]*circularBufferEntry, l) + ce.index = make(map[string]*indexEntry, l/estimatedExemplarsPerSeries) + ce.nextIndex = 0 + + // Replay as many entries as needed, starting with oldest first. + count := int64(len(oldBuffer)) + if l < count { + count = l + } + + migrated := 0 + + if l > 0 && len(oldBuffer) > 0 { + // Rewind previous next index by count with wrap-around. + // This math is essentially looking at nextIndex, where we would write the next exemplar to, + // and find the index in the old exemplar buffer that we should start migrating exemplars from. + // This way we don't migrate exemplars that would just be overwritten when migrating later exemplars. + startIndex := (oldNextIndex - count + int64(len(oldBuffer))) % int64(len(oldBuffer)) + + for i := int64(0); i < count; i++ { + idx := (startIndex + i) % int64(len(oldBuffer)) + if entry := oldBuffer[idx]; entry != nil { + ce.migrate(entry) + migrated++ + } + } + } + + ce.computeMetrics() + ce.metrics.maxExemplars.Set(float64(l)) + + return migrated +} + +// migrate is like AddExemplar but reuses existing structs. Expected to be called in batch and requires +// external lock and does not compute metrics. +func (ce *CircularExemplarStorage) migrate(entry *circularBufferEntry) { + var buf [1024]byte + seriesLabels := entry.ref.seriesLabels.Bytes(buf[:]) + + idx, ok := ce.index[string(seriesLabels)] + if !ok { + idx = entry.ref + idx.oldest = ce.nextIndex + ce.index[string(seriesLabels)] = idx + } else { + entry.ref = idx + ce.exemplars[idx.newest].next = ce.nextIndex + } + idx.newest = ce.nextIndex + + entry.next = noExemplar + ce.exemplars[ce.nextIndex] = entry + + ce.nextIndex = (ce.nextIndex + 1) % len(ce.exemplars) +} + +func (ce *CircularExemplarStorage) AddExemplar(l labels.Labels, e exemplar.Exemplar) error { + if len(ce.exemplars) <= 0 { + return storage.ErrExemplarsDisabled + } + + var buf [1024]byte + seriesLabels := l.Bytes(buf[:]) + + // TODO(bwplotka): This lock can lock all scrapers, there might high contention on this on scale. + // Optimize by moving the lock to be per series (& benchmark it). + ce.lock.Lock() + defer ce.lock.Unlock() + + err := ce.validateExemplar(seriesLabels, e, true) + if err != nil { + if err == storage.ErrDuplicateExemplar { + // Duplicate exemplar, noop. + return nil + } + return err + } + + _, ok := ce.index[string(seriesLabels)] + if !ok { + ce.index[string(seriesLabels)] = &indexEntry{oldest: ce.nextIndex, seriesLabels: l} + } else { + ce.exemplars[ce.index[string(seriesLabels)].newest].next = ce.nextIndex + } + + if prev := ce.exemplars[ce.nextIndex]; prev == nil { + ce.exemplars[ce.nextIndex] = &circularBufferEntry{} + } else { + // There exists exemplar already on this ce.nextIndex entry, drop it, to make place + // for others. + var buf [1024]byte + prevLabels := prev.ref.seriesLabels.Bytes(buf[:]) + if prev.next == noExemplar { + // Last item for this series, remove index entry. + delete(ce.index, string(prevLabels)) + } else { + ce.index[string(prevLabels)].oldest = prev.next + } + } + + // Default the next value to -1 (which we use to detect that we've iterated through all exemplars for a series in Select) + // since this is the first exemplar stored for this series. + ce.exemplars[ce.nextIndex].next = noExemplar + ce.exemplars[ce.nextIndex].exemplar = e + ce.exemplars[ce.nextIndex].ref = ce.index[string(seriesLabels)] + ce.index[string(seriesLabels)].newest = ce.nextIndex + + ce.nextIndex = (ce.nextIndex + 1) % len(ce.exemplars) + + ce.metrics.exemplarsAppended.Inc() + ce.computeMetrics() + return nil +} + +func (ce *CircularExemplarStorage) computeMetrics() { + ce.metrics.seriesWithExemplarsInStorage.Set(float64(len(ce.index))) + + if len(ce.exemplars) == 0 { + ce.metrics.exemplarsInStorage.Set(float64(0)) + ce.metrics.lastExemplarsTs.Set(float64(0)) + return + } + + if next := ce.exemplars[ce.nextIndex]; next != nil { + ce.metrics.exemplarsInStorage.Set(float64(len(ce.exemplars))) + ce.metrics.lastExemplarsTs.Set(float64(next.exemplar.Ts) / 1000) + return + } + + // We did not yet fill the buffer. + ce.metrics.exemplarsInStorage.Set(float64(ce.nextIndex)) + if ce.exemplars[0] != nil { + ce.metrics.lastExemplarsTs.Set(float64(ce.exemplars[0].exemplar.Ts) / 1000) + } +} + +// IterateExemplars iterates through all the exemplars from oldest to newest appended and calls +// the given function on all of them till the end (or) till the first function call that returns an error. +func (ce *CircularExemplarStorage) IterateExemplars(f func(seriesLabels labels.Labels, e exemplar.Exemplar) error) error { + ce.lock.RLock() + defer ce.lock.RUnlock() + + idx := ce.nextIndex + l := len(ce.exemplars) + for i := 0; i < l; i, idx = i+1, (idx+1)%l { + if ce.exemplars[idx] == nil { + continue + } + err := f(ce.exemplars[idx].ref.seriesLabels, ce.exemplars[idx].exemplar) + if err != nil { + return err + } + } + return nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/goversion/goversion.go b/vendor/github.com/prometheus/prometheus/tsdb/goversion/goversion.go new file mode 100644 index 00000000000..02da2639d6c --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/goversion/goversion.go @@ -0,0 +1,20 @@ +// Copyright 2017 The Prometheus Authors +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//go:build go1.12 +// +build go1.12 + +// Package goversion enforces the go version supported by the tsdb module. +package goversion + +const _SoftwareRequiresGOVERSION1_12 = uint8(0) diff --git a/vendor/github.com/prometheus/prometheus/tsdb/goversion/init.go b/vendor/github.com/prometheus/prometheus/tsdb/goversion/init.go new file mode 100644 index 00000000000..dd15e1f7afd --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/goversion/init.go @@ -0,0 +1,17 @@ +// Copyright 2017 The Prometheus Authors +// 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 goversion + +// This will fail to compile if the Go runtime version isn't >= 1.12. +var _ = _SoftwareRequiresGOVERSION1_12 diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head.go b/vendor/github.com/prometheus/prometheus/tsdb/head.go new file mode 100644 index 00000000000..5663e3cf90d --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head.go @@ -0,0 +1,1678 @@ +// Copyright 2017 The Prometheus Authors +// 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 tsdb + +import ( + "fmt" + "io" + "math" + "path/filepath" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "go.uber.org/atomic" + + "github.com/prometheus/prometheus/config" + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/record" + "github.com/prometheus/prometheus/tsdb/tombstones" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/prometheus/prometheus/tsdb/wal" +) + +var ( + // ErrInvalidSample is returned if an appended sample is not valid and can't + // be ingested. + ErrInvalidSample = errors.New("invalid sample") + // ErrInvalidExemplar is returned if an appended exemplar is not valid and can't + // be ingested. + ErrInvalidExemplar = errors.New("invalid exemplar") + // ErrAppenderClosed is returned if an appender has already be successfully + // rolled back or committed. + ErrAppenderClosed = errors.New("appender closed") + + // defaultIsolationDisabled is true if isolation is disabled by default. + defaultIsolationDisabled = false +) + +// Head handles reads and writes of time series data within a time window. +type Head struct { + chunkRange atomic.Int64 + numSeries atomic.Uint64 + minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head. + minValidTime atomic.Int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block. + lastWALTruncationTime atomic.Int64 + lastMemoryTruncationTime atomic.Int64 + lastSeriesID atomic.Uint64 + + metrics *headMetrics + opts *HeadOptions + wal *wal.WAL + exemplarMetrics *ExemplarMetrics + exemplars ExemplarStorage + logger log.Logger + appendPool sync.Pool + exemplarsPool sync.Pool + seriesPool sync.Pool + bytesPool sync.Pool + memChunkPool sync.Pool + + // All series addressable by their ID or hash. + series *stripeSeries + + deletedMtx sync.Mutex + deleted map[chunks.HeadSeriesRef]int // Deleted series, and what WAL segment they must be kept until. + + postings *index.MemPostings // Postings lists for terms. + + tombstones *tombstones.MemTombstones + + iso *isolation + + cardinalityMutex sync.Mutex + cardinalityCache *index.PostingsStats // Posting stats cache which will expire after 30sec. + lastPostingsStatsCall time.Duration // Last posting stats call (PostingsCardinalityStats()) time for caching. + + // chunkDiskMapper is used to write and read Head chunks to/from disk. + chunkDiskMapper *chunks.ChunkDiskMapper + + chunkSnapshotMtx sync.Mutex + + closedMtx sync.Mutex + closed bool + + stats *HeadStats + reg prometheus.Registerer + + memTruncationInProcess atomic.Bool +} + +type ExemplarStorage interface { + storage.ExemplarQueryable + AddExemplar(labels.Labels, exemplar.Exemplar) error + ValidateExemplar(labels.Labels, exemplar.Exemplar) error + IterateExemplars(f func(seriesLabels labels.Labels, e exemplar.Exemplar) error) error +} + +// HeadOptions are parameters for the Head block. +type HeadOptions struct { + // Runtime reloadable option. At the top of the struct for 32 bit OS: + // https://pkg.go.dev/sync/atomic#pkg-note-BUG + MaxExemplars atomic.Int64 + + ChunkRange int64 + // ChunkDirRoot is the parent directory of the chunks directory. + ChunkDirRoot string + ChunkPool chunkenc.Pool + ChunkWriteBufferSize int + ChunkWriteQueueSize int + + // StripeSize sets the number of entries in the hash map, it must be a power of 2. + // A larger StripeSize will allocate more memory up-front, but will increase performance when handling a large number of series. + // A smaller StripeSize reduces the memory allocated, but can decrease performance with large number of series. + StripeSize int + SeriesCallback SeriesLifecycleCallback + EnableExemplarStorage bool + EnableMemorySnapshotOnShutdown bool + + IsolationDisabled bool +} + +func DefaultHeadOptions() *HeadOptions { + return &HeadOptions{ + ChunkRange: DefaultBlockDuration, + ChunkDirRoot: "", + ChunkPool: chunkenc.NewPool(), + ChunkWriteBufferSize: chunks.DefaultWriteBufferSize, + ChunkWriteQueueSize: chunks.DefaultWriteQueueSize, + StripeSize: DefaultStripeSize, + SeriesCallback: &noopSeriesLifecycleCallback{}, + IsolationDisabled: defaultIsolationDisabled, + } +} + +// SeriesLifecycleCallback specifies a list of callbacks that will be called during a lifecycle of a series. +// It is always a no-op in Prometheus and mainly meant for external users who import TSDB. +// All the callbacks should be safe to be called concurrently. +// It is up to the user to implement soft or hard consistency by making the callbacks +// atomic or non-atomic. Atomic callbacks can cause degradation performance. +type SeriesLifecycleCallback interface { + // PreCreation is called before creating a series to indicate if the series can be created. + // A non nil error means the series should not be created. + PreCreation(labels.Labels) error + // PostCreation is called after creating a series to indicate a creation of series. + PostCreation(labels.Labels) + // PostDeletion is called after deletion of series. + PostDeletion(...labels.Labels) +} + +// NewHead opens the head block in dir. +func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, opts *HeadOptions, stats *HeadStats) (*Head, error) { + var err error + if l == nil { + l = log.NewNopLogger() + } + if opts.ChunkRange < 1 { + return nil, errors.Errorf("invalid chunk range %d", opts.ChunkRange) + } + if opts.SeriesCallback == nil { + opts.SeriesCallback = &noopSeriesLifecycleCallback{} + } + + if stats == nil { + stats = NewHeadStats() + } + + if !opts.EnableExemplarStorage { + opts.MaxExemplars.Store(0) + } + + h := &Head{ + wal: wal, + logger: l, + opts: opts, + memChunkPool: sync.Pool{ + New: func() interface{} { + return &memChunk{} + }, + }, + stats: stats, + reg: r, + } + if err := h.resetInMemoryState(); err != nil { + return nil, err + } + h.metrics = newHeadMetrics(h, r) + + if opts.ChunkPool == nil { + opts.ChunkPool = chunkenc.NewPool() + } + + h.chunkDiskMapper, err = chunks.NewChunkDiskMapper( + r, + mmappedChunksDir(opts.ChunkDirRoot), + opts.ChunkPool, + opts.ChunkWriteBufferSize, + opts.ChunkWriteQueueSize, + ) + if err != nil { + return nil, err + } + + return h, nil +} + +func (h *Head) resetInMemoryState() error { + var err error + var em *ExemplarMetrics + if h.exemplars != nil { + ce, ok := h.exemplars.(*CircularExemplarStorage) + if ok { + em = ce.metrics + } + } + if em == nil { + em = NewExemplarMetrics(h.reg) + } + es, err := NewCircularExemplarStorage(h.opts.MaxExemplars.Load(), em) + if err != nil { + return err + } + + h.iso = newIsolation(h.opts.IsolationDisabled) + + h.exemplarMetrics = em + h.exemplars = es + h.series = newStripeSeries(h.opts.StripeSize, h.opts.SeriesCallback) + h.postings = index.NewUnorderedMemPostings() + h.tombstones = tombstones.NewMemTombstones() + h.deleted = map[chunks.HeadSeriesRef]int{} + h.chunkRange.Store(h.opts.ChunkRange) + h.minTime.Store(math.MaxInt64) + h.maxTime.Store(math.MinInt64) + h.lastWALTruncationTime.Store(math.MinInt64) + h.lastMemoryTruncationTime.Store(math.MinInt64) + return nil +} + +type headMetrics struct { + activeAppenders prometheus.Gauge + series prometheus.GaugeFunc + seriesCreated prometheus.Counter + seriesRemoved prometheus.Counter + seriesNotFound prometheus.Counter + chunks prometheus.Gauge + chunksCreated prometheus.Counter + chunksRemoved prometheus.Counter + gcDuration prometheus.Summary + samplesAppended prometheus.Counter + outOfBoundSamples prometheus.Counter + outOfOrderSamples prometheus.Counter + walTruncateDuration prometheus.Summary + walCorruptionsTotal prometheus.Counter + walTotalReplayDuration prometheus.Gauge + headTruncateFail prometheus.Counter + headTruncateTotal prometheus.Counter + checkpointDeleteFail prometheus.Counter + checkpointDeleteTotal prometheus.Counter + checkpointCreationFail prometheus.Counter + checkpointCreationTotal prometheus.Counter + mmapChunkCorruptionTotal prometheus.Counter + snapshotReplayErrorTotal prometheus.Counter // Will be either 0 or 1. +} + +func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics { + m := &headMetrics{ + activeAppenders: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_active_appenders", + Help: "Number of currently active appender transactions", + }), + series: prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_series", + Help: "Total number of series in the head block.", + }, func() float64 { + return float64(h.NumSeries()) + }), + seriesCreated: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_series_created_total", + Help: "Total number of series created in the head", + }), + seriesRemoved: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_series_removed_total", + Help: "Total number of series removed in the head", + }), + seriesNotFound: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_series_not_found_total", + Help: "Total number of requests for series that were not found.", + }), + chunks: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_chunks", + Help: "Total number of chunks in the head block.", + }), + chunksCreated: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_chunks_created_total", + Help: "Total number of chunks created in the head", + }), + chunksRemoved: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_chunks_removed_total", + Help: "Total number of chunks removed in the head", + }), + gcDuration: prometheus.NewSummary(prometheus.SummaryOpts{ + Name: "prometheus_tsdb_head_gc_duration_seconds", + Help: "Runtime of garbage collection in the head block.", + }), + walTruncateDuration: prometheus.NewSummary(prometheus.SummaryOpts{ + Name: "prometheus_tsdb_wal_truncate_duration_seconds", + Help: "Duration of WAL truncation.", + }), + walCorruptionsTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_wal_corruptions_total", + Help: "Total number of WAL corruptions.", + }), + walTotalReplayDuration: prometheus.NewGauge(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_data_replay_duration_seconds", + Help: "Time taken to replay the data on disk.", + }), + samplesAppended: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_samples_appended_total", + Help: "Total number of appended samples.", + }), + outOfBoundSamples: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_out_of_bound_samples_total", + Help: "Total number of out of bound samples ingestion failed attempts.", + }), + outOfOrderSamples: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_out_of_order_samples_total", + Help: "Total number of out of order samples ingestion failed attempts.", + }), + headTruncateFail: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_truncations_failed_total", + Help: "Total number of head truncations that failed.", + }), + headTruncateTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_head_truncations_total", + Help: "Total number of head truncations attempted.", + }), + checkpointDeleteFail: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_checkpoint_deletions_failed_total", + Help: "Total number of checkpoint deletions that failed.", + }), + checkpointDeleteTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_checkpoint_deletions_total", + Help: "Total number of checkpoint deletions attempted.", + }), + checkpointCreationFail: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_checkpoint_creations_failed_total", + Help: "Total number of checkpoint creations that failed.", + }), + checkpointCreationTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_checkpoint_creations_total", + Help: "Total number of checkpoint creations attempted.", + }), + mmapChunkCorruptionTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_mmap_chunk_corruptions_total", + Help: "Total number of memory-mapped chunk corruptions.", + }), + snapshotReplayErrorTotal: prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_snapshot_replay_error_total", + Help: "Total number snapshot replays that failed.", + }), + } + + if r != nil { + r.MustRegister( + m.activeAppenders, + m.series, + m.chunks, + m.chunksCreated, + m.chunksRemoved, + m.seriesCreated, + m.seriesRemoved, + m.seriesNotFound, + m.gcDuration, + m.walTruncateDuration, + m.walCorruptionsTotal, + m.walTotalReplayDuration, + m.samplesAppended, + m.outOfBoundSamples, + m.outOfOrderSamples, + m.headTruncateFail, + m.headTruncateTotal, + m.checkpointDeleteFail, + m.checkpointDeleteTotal, + m.checkpointCreationFail, + m.checkpointCreationTotal, + m.mmapChunkCorruptionTotal, + m.snapshotReplayErrorTotal, + // Metrics bound to functions and not needed in tests + // can be created and registered on the spot. + prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_max_time", + Help: "Maximum timestamp of the head block. The unit is decided by the library consumer.", + }, func() float64 { + return float64(h.MaxTime()) + }), + prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_head_min_time", + Help: "Minimum time bound of the head block. The unit is decided by the library consumer.", + }, func() float64 { + return float64(h.MinTime()) + }), + prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_isolation_low_watermark", + Help: "The lowest TSDB append ID that is still referenced.", + }, func() float64 { + return float64(h.iso.lowWatermark()) + }), + prometheus.NewGaugeFunc(prometheus.GaugeOpts{ + Name: "prometheus_tsdb_isolation_high_watermark", + Help: "The highest TSDB append ID that has been given out.", + }, func() float64 { + return float64(h.iso.lastAppendID()) + }), + ) + } + return m +} + +func mmappedChunksDir(dir string) string { return filepath.Join(dir, "chunks_head") } + +// HeadStats are the statistics for the head component of the DB. +type HeadStats struct { + WALReplayStatus *WALReplayStatus +} + +// NewHeadStats returns a new HeadStats object. +func NewHeadStats() *HeadStats { + return &HeadStats{ + WALReplayStatus: &WALReplayStatus{}, + } +} + +// WALReplayStatus contains status information about the WAL replay. +type WALReplayStatus struct { + sync.RWMutex + Min int + Max int + Current int +} + +// GetWALReplayStatus returns the WAL replay status information. +func (s *WALReplayStatus) GetWALReplayStatus() WALReplayStatus { + s.RLock() + defer s.RUnlock() + + return WALReplayStatus{ + Min: s.Min, + Max: s.Max, + Current: s.Current, + } +} + +const cardinalityCacheExpirationTime = time.Duration(30) * time.Second + +// Init loads data from the write ahead log and prepares the head for writes. +// It should be called before using an appender so that it +// limits the ingested samples to the head min valid time. +func (h *Head) Init(minValidTime int64) error { + h.minValidTime.Store(minValidTime) + defer func() { + h.postings.EnsureOrder() + }() + defer h.gc() // After loading the wal remove the obsolete data from the head. + defer func() { + // Loading of m-mapped chunks and snapshot can make the mint of the Head + // to go below minValidTime. + if h.MinTime() < h.minValidTime.Load() { + h.minTime.Store(h.minValidTime.Load()) + } + }() + + level.Info(h.logger).Log("msg", "Replaying on-disk memory mappable chunks if any") + start := time.Now() + + snapIdx, snapOffset := -1, 0 + refSeries := make(map[chunks.HeadSeriesRef]*memSeries) + + if h.opts.EnableMemorySnapshotOnShutdown { + level.Info(h.logger).Log("msg", "Chunk snapshot is enabled, replaying from the snapshot") + var err error + snapIdx, snapOffset, refSeries, err = h.loadChunkSnapshot() + if err != nil { + snapIdx, snapOffset = -1, 0 + refSeries = make(map[chunks.HeadSeriesRef]*memSeries) + + h.metrics.snapshotReplayErrorTotal.Inc() + level.Error(h.logger).Log("msg", "Failed to load chunk snapshot", "err", err) + // We clear the partially loaded data to replay fresh from the WAL. + if err := h.resetInMemoryState(); err != nil { + return err + } + } + level.Info(h.logger).Log("msg", "Chunk snapshot loading time", "duration", time.Since(start).String()) + } + + mmapChunkReplayStart := time.Now() + mmappedChunks, err := h.loadMmappedChunks(refSeries) + if err != nil { + level.Error(h.logger).Log("msg", "Loading on-disk chunks failed", "err", err) + if _, ok := errors.Cause(err).(*chunks.CorruptionErr); ok { + h.metrics.mmapChunkCorruptionTotal.Inc() + } + + // Discard snapshot data since we need to replay the WAL for the missed m-map chunks data. + snapIdx, snapOffset = -1, 0 + + // If this fails, data will be recovered from WAL. + // Hence we wont lose any data (given WAL is not corrupt). + mmappedChunks, err = h.removeCorruptedMmappedChunks(err) + if err != nil { + return err + } + } + + level.Info(h.logger).Log("msg", "On-disk memory mappable chunks replay completed", "duration", time.Since(mmapChunkReplayStart).String()) + if h.wal == nil { + level.Info(h.logger).Log("msg", "WAL not found") + return nil + } + + level.Info(h.logger).Log("msg", "Replaying WAL, this may take a while") + + checkpointReplayStart := time.Now() + // Backfill the checkpoint first if it exists. + dir, startFrom, err := wal.LastCheckpoint(h.wal.Dir()) + if err != nil && err != record.ErrNotFound { + return errors.Wrap(err, "find last checkpoint") + } + + // Find the last segment. + _, endAt, e := wal.Segments(h.wal.Dir()) + if e != nil { + return errors.Wrap(e, "finding WAL segments") + } + + h.startWALReplayStatus(startFrom, endAt) + + multiRef := map[chunks.HeadSeriesRef]chunks.HeadSeriesRef{} + if err == nil && startFrom >= snapIdx { + sr, err := wal.NewSegmentsReader(dir) + if err != nil { + return errors.Wrap(err, "open checkpoint") + } + defer func() { + if err := sr.Close(); err != nil { + level.Warn(h.logger).Log("msg", "Error while closing the wal segments reader", "err", err) + } + }() + + // A corrupted checkpoint is a hard error for now and requires user + // intervention. There's likely little data that can be recovered anyway. + if err := h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks); err != nil { + return errors.Wrap(err, "backfill checkpoint") + } + h.updateWALReplayStatusRead(startFrom) + startFrom++ + level.Info(h.logger).Log("msg", "WAL checkpoint loaded") + } + checkpointReplayDuration := time.Since(checkpointReplayStart) + + walReplayStart := time.Now() + + if snapIdx > startFrom { + startFrom = snapIdx + } + // Backfill segments from the most recent checkpoint onwards. + for i := startFrom; i <= endAt; i++ { + s, err := wal.OpenReadSegment(wal.SegmentName(h.wal.Dir(), i)) + if err != nil { + return errors.Wrap(err, fmt.Sprintf("open WAL segment: %d", i)) + } + + offset := 0 + if i == snapIdx { + offset = snapOffset + } + sr, err := wal.NewSegmentBufReaderWithOffset(offset, s) + if errors.Cause(err) == io.EOF { + // File does not exist. + continue + } + if err != nil { + return errors.Wrapf(err, "segment reader (offset=%d)", offset) + } + err = h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks) + if err := sr.Close(); err != nil { + level.Warn(h.logger).Log("msg", "Error while closing the wal segments reader", "err", err) + } + if err != nil { + return err + } + level.Info(h.logger).Log("msg", "WAL segment loaded", "segment", i, "maxSegment", endAt) + h.updateWALReplayStatusRead(i) + } + + walReplayDuration := time.Since(start) + h.metrics.walTotalReplayDuration.Set(walReplayDuration.Seconds()) + level.Info(h.logger).Log( + "msg", "WAL replay completed", + "checkpoint_replay_duration", checkpointReplayDuration.String(), + "wal_replay_duration", time.Since(walReplayStart).String(), + "total_replay_duration", walReplayDuration.String(), + ) + + return nil +} + +func (h *Head) loadMmappedChunks(refSeries map[chunks.HeadSeriesRef]*memSeries) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) { + mmappedChunks := map[chunks.HeadSeriesRef][]*mmappedChunk{} + if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef chunks.HeadSeriesRef, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error { + if maxt < h.minValidTime.Load() { + return nil + } + ms, ok := refSeries[seriesRef] + if !ok { + slice := mmappedChunks[seriesRef] + if len(slice) > 0 && slice[len(slice)-1].maxTime >= mint { + return errors.Errorf("out of sequence m-mapped chunk for series ref %d, last chunk: [%d, %d], new: [%d, %d]", + seriesRef, slice[len(slice)-1].minTime, slice[len(slice)-1].maxTime, mint, maxt) + } + slice = append(slice, &mmappedChunk{ + ref: chunkRef, + minTime: mint, + maxTime: maxt, + numSamples: numSamples, + }) + mmappedChunks[seriesRef] = slice + return nil + } + + if len(ms.mmappedChunks) > 0 && ms.mmappedChunks[len(ms.mmappedChunks)-1].maxTime >= mint { + return errors.Errorf("out of sequence m-mapped chunk for series ref %d, last chunk: [%d, %d], new: [%d, %d]", + seriesRef, ms.mmappedChunks[len(ms.mmappedChunks)-1].minTime, ms.mmappedChunks[len(ms.mmappedChunks)-1].maxTime, + mint, maxt) + } + + h.metrics.chunks.Inc() + h.metrics.chunksCreated.Inc() + ms.mmappedChunks = append(ms.mmappedChunks, &mmappedChunk{ + ref: chunkRef, + minTime: mint, + maxTime: maxt, + numSamples: numSamples, + }) + h.updateMinMaxTime(mint, maxt) + if ms.headChunk != nil && maxt >= ms.headChunk.minTime { + // The head chunk was completed and was m-mapped after taking the snapshot. + // Hence remove this chunk. + ms.nextAt = 0 + ms.headChunk = nil + ms.app = nil + } + return nil + }); err != nil { + return nil, errors.Wrap(err, "iterate on on-disk chunks") + } + return mmappedChunks, nil +} + +// removeCorruptedMmappedChunks attempts to delete the corrupted mmapped chunks and if it fails, it clears all the previously +// loaded mmapped chunks. +func (h *Head) removeCorruptedMmappedChunks(err error) (map[chunks.HeadSeriesRef][]*mmappedChunk, error) { + // We never want to preserve the in-memory series from snapshots if we are repairing m-map chunks. + if err := h.resetInMemoryState(); err != nil { + return nil, err + } + + level.Info(h.logger).Log("msg", "Deleting mmapped chunk files") + + if err := h.chunkDiskMapper.DeleteCorrupted(err); err != nil { + level.Info(h.logger).Log("msg", "Deletion of corrupted mmap chunk files failed, discarding chunk files completely", "err", err) + if err := h.chunkDiskMapper.Truncate(math.MaxInt64); err != nil { + level.Error(h.logger).Log("msg", "Deletion of all mmap chunk files failed", "err", err) + } + return map[chunks.HeadSeriesRef][]*mmappedChunk{}, nil + } + + level.Info(h.logger).Log("msg", "Deletion of mmap chunk files successful, reattempting m-mapping the on-disk chunks") + mmappedChunks, err := h.loadMmappedChunks(make(map[chunks.HeadSeriesRef]*memSeries)) + if err != nil { + level.Error(h.logger).Log("msg", "Loading on-disk chunks failed, discarding chunk files completely", "err", err) + if err := h.chunkDiskMapper.Truncate(math.MaxInt64); err != nil { + level.Error(h.logger).Log("msg", "Deletion of all mmap chunk files failed after failed loading", "err", err) + } + mmappedChunks = map[chunks.HeadSeriesRef][]*mmappedChunk{} + } + + return mmappedChunks, nil +} + +func (h *Head) ApplyConfig(cfg *config.Config) error { + if !h.opts.EnableExemplarStorage { + return nil + } + + // Head uses opts.MaxExemplars in combination with opts.EnableExemplarStorage + // to decide if it should pass exemplars along to its exemplar storage, so we + // need to update opts.MaxExemplars here. + prevSize := h.opts.MaxExemplars.Load() + h.opts.MaxExemplars.Store(cfg.StorageConfig.ExemplarsConfig.MaxExemplars) + newSize := h.opts.MaxExemplars.Load() + + if prevSize == newSize { + return nil + } + + migrated := h.exemplars.(*CircularExemplarStorage).Resize(newSize) + level.Info(h.logger).Log("msg", "Exemplar storage resized", "from", prevSize, "to", newSize, "migrated", migrated) + return nil +} + +// PostingsCardinalityStats returns top 10 highest cardinality stats By label and value names. +func (h *Head) PostingsCardinalityStats(statsByLabelName string) *index.PostingsStats { + h.cardinalityMutex.Lock() + defer h.cardinalityMutex.Unlock() + currentTime := time.Duration(time.Now().Unix()) * time.Second + seconds := currentTime - h.lastPostingsStatsCall + if seconds > cardinalityCacheExpirationTime { + h.cardinalityCache = nil + } + if h.cardinalityCache != nil { + return h.cardinalityCache + } + h.cardinalityCache = h.postings.Stats(statsByLabelName) + h.lastPostingsStatsCall = time.Duration(time.Now().Unix()) * time.Second + + return h.cardinalityCache +} + +func (h *Head) updateMinMaxTime(mint, maxt int64) { + for { + lt := h.MinTime() + if mint >= lt { + break + } + if h.minTime.CAS(lt, mint) { + break + } + } + for { + ht := h.MaxTime() + if maxt <= ht { + break + } + if h.maxTime.CAS(ht, maxt) { + break + } + } +} + +// SetMinValidTime sets the minimum timestamp the head can ingest. +func (h *Head) SetMinValidTime(minValidTime int64) { + h.minValidTime.Store(minValidTime) +} + +// Truncate removes old data before mint from the head and WAL. +func (h *Head) Truncate(mint int64) (err error) { + initialize := h.MinTime() == math.MaxInt64 + if err := h.truncateMemory(mint); err != nil { + return err + } + if initialize { + return nil + } + return h.truncateWAL(mint) +} + +// OverlapsClosedInterval returns true if the head overlaps [mint, maxt]. +func (h *Head) OverlapsClosedInterval(mint, maxt int64) bool { + return h.MinTime() <= maxt && mint <= h.MaxTime() +} + +// truncateMemory removes old data before mint from the head. +func (h *Head) truncateMemory(mint int64) (err error) { + h.chunkSnapshotMtx.Lock() + defer h.chunkSnapshotMtx.Unlock() + + defer func() { + if err != nil { + h.metrics.headTruncateFail.Inc() + } + }() + + initialize := h.MinTime() == math.MaxInt64 + + if h.MinTime() >= mint && !initialize { + return nil + } + + // The order of these two Store() should not be changed, + // i.e. truncation time is set before in-process boolean. + h.lastMemoryTruncationTime.Store(mint) + h.memTruncationInProcess.Store(true) + defer h.memTruncationInProcess.Store(false) + + // We wait for pending queries to end that overlap with this truncation. + if !initialize { + h.WaitForPendingReadersInTimeRange(h.MinTime(), mint) + } + + h.minTime.Store(mint) + h.minValidTime.Store(mint) + + // Ensure that max time is at least as high as min time. + for h.MaxTime() < mint { + h.maxTime.CAS(h.MaxTime(), mint) + } + + // This was an initial call to Truncate after loading blocks on startup. + // We haven't read back the WAL yet, so do not attempt to truncate it. + if initialize { + return nil + } + + h.metrics.headTruncateTotal.Inc() + start := time.Now() + + actualMint := h.gc() + level.Info(h.logger).Log("msg", "Head GC completed", "duration", time.Since(start)) + h.metrics.gcDuration.Observe(time.Since(start).Seconds()) + if actualMint > h.minTime.Load() { + // The actual mint of the Head is higher than the one asked to truncate. + appendableMinValidTime := h.appendableMinValidTime() + if actualMint < appendableMinValidTime { + h.minTime.Store(actualMint) + h.minValidTime.Store(actualMint) + } else { + // The actual min time is in the appendable window. + // So we set the mint to the appendableMinValidTime. + h.minTime.Store(appendableMinValidTime) + h.minValidTime.Store(appendableMinValidTime) + } + } + + // Truncate the chunk m-mapper. + if err := h.chunkDiskMapper.Truncate(mint); err != nil { + return errors.Wrap(err, "truncate chunks.HeadReadWriter") + } + return nil +} + +// WaitForPendingReadersInTimeRange waits for queries overlapping with given range to finish querying. +// The query timeout limits the max wait time of this function implicitly. +// The mint is inclusive and maxt is the truncation time hence exclusive. +func (h *Head) WaitForPendingReadersInTimeRange(mint, maxt int64) { + maxt-- // Making it inclusive before checking overlaps. + overlaps := func() bool { + o := false + h.iso.TraverseOpenReads(func(s *isolationState) bool { + if s.mint <= maxt && mint <= s.maxt { + // Overlaps with the truncation range. + o = true + return false + } + return true + }) + return o + } + for overlaps() { + time.Sleep(500 * time.Millisecond) + } +} + +// IsQuerierCollidingWithTruncation returns if the current querier needs to be closed and if a new querier +// has to be created. In the latter case, the method also returns the new mint to be used for creating the +// new range head and the new querier. This methods helps preventing races with the truncation of in-memory data. +// +// NOTE: The querier should already be taken before calling this. +func (h *Head) IsQuerierCollidingWithTruncation(querierMint, querierMaxt int64) (shouldClose, getNew bool, newMint int64) { + if !h.memTruncationInProcess.Load() { + return false, false, 0 + } + // Head truncation is in process. It also means that the block that was + // created for this truncation range is also available. + // Check if we took a querier that overlaps with this truncation. + memTruncTime := h.lastMemoryTruncationTime.Load() + if querierMaxt < memTruncTime { + // Head compaction has happened and this time range is being truncated. + // This query doesn't overlap with the Head any longer. + // We should close this querier to avoid races and the data would be + // available with the blocks below. + // Cases: + // 1. |------truncation------| + // |---query---| + // 2. |------truncation------| + // |---query---| + return true, false, 0 + } + if querierMint < memTruncTime { + // The truncation time is not same as head mint that we saw above but the + // query still overlaps with the Head. + // The truncation started after we got the querier. So it is not safe + // to use this querier and/or might block truncation. We should get + // a new querier for the new Head range while remaining will be available + // in the blocks below. + // Case: + // |------truncation------| + // |----query----| + // Turns into + // |------truncation------| + // |---qu---| + return true, true, memTruncTime + } + + // Other case is this, which is a no-op + // |------truncation------| + // |---query---| + return false, false, 0 +} + +// truncateWAL removes old data before mint from the WAL. +func (h *Head) truncateWAL(mint int64) error { + h.chunkSnapshotMtx.Lock() + defer h.chunkSnapshotMtx.Unlock() + + if h.wal == nil || mint <= h.lastWALTruncationTime.Load() { + return nil + } + start := time.Now() + h.lastWALTruncationTime.Store(mint) + + first, last, err := wal.Segments(h.wal.Dir()) + if err != nil { + return errors.Wrap(err, "get segment range") + } + // Start a new segment, so low ingestion volume TSDB don't have more WAL than + // needed. + if err := h.wal.NextSegment(); err != nil { + return errors.Wrap(err, "next segment") + } + last-- // Never consider last segment for checkpoint. + if last < 0 { + return nil // no segments yet. + } + // The lower two thirds of segments should contain mostly obsolete samples. + // If we have less than two segments, it's not worth checkpointing yet. + // With the default 2h blocks, this will keeping up to around 3h worth + // of WAL segments. + last = first + (last-first)*2/3 + if last <= first { + return nil + } + + keep := func(id chunks.HeadSeriesRef) bool { + if h.series.getByID(id) != nil { + return true + } + h.deletedMtx.Lock() + _, ok := h.deleted[id] + h.deletedMtx.Unlock() + return ok + } + h.metrics.checkpointCreationTotal.Inc() + if _, err = wal.Checkpoint(h.logger, h.wal, first, last, keep, mint); err != nil { + h.metrics.checkpointCreationFail.Inc() + if _, ok := errors.Cause(err).(*wal.CorruptionErr); ok { + h.metrics.walCorruptionsTotal.Inc() + } + return errors.Wrap(err, "create checkpoint") + } + if err := h.wal.Truncate(last + 1); err != nil { + // If truncating fails, we'll just try again at the next checkpoint. + // Leftover segments will just be ignored in the future if there's a checkpoint + // that supersedes them. + level.Error(h.logger).Log("msg", "truncating segments failed", "err", err) + } + + // The checkpoint is written and segments before it is truncated, so we no + // longer need to track deleted series that are before it. + h.deletedMtx.Lock() + for ref, segment := range h.deleted { + if segment < first { + delete(h.deleted, ref) + } + } + h.deletedMtx.Unlock() + + h.metrics.checkpointDeleteTotal.Inc() + if err := wal.DeleteCheckpoints(h.wal.Dir(), last); err != nil { + // Leftover old checkpoints do not cause problems down the line beyond + // occupying disk space. + // They will just be ignored since a higher checkpoint exists. + level.Error(h.logger).Log("msg", "delete old checkpoints", "err", err) + h.metrics.checkpointDeleteFail.Inc() + } + h.metrics.walTruncateDuration.Observe(time.Since(start).Seconds()) + + level.Info(h.logger).Log("msg", "WAL checkpoint complete", + "first", first, "last", last, "duration", time.Since(start)) + + return nil +} + +type Stats struct { + NumSeries uint64 + MinTime, MaxTime int64 + IndexPostingStats *index.PostingsStats +} + +// Stats returns important current HEAD statistics. Note that it is expensive to +// calculate these. +func (h *Head) Stats(statsByLabelName string) *Stats { + return &Stats{ + NumSeries: h.NumSeries(), + MaxTime: h.MaxTime(), + MinTime: h.MinTime(), + IndexPostingStats: h.PostingsCardinalityStats(statsByLabelName), + } +} + +// RangeHead allows querying Head via an IndexReader, ChunkReader and tombstones.Reader +// but only within a restricted range. Used for queries and compactions. +type RangeHead struct { + head *Head + mint, maxt int64 +} + +// NewRangeHead returns a *RangeHead. +// There are no restrictions on mint/maxt. +func NewRangeHead(head *Head, mint, maxt int64) *RangeHead { + return &RangeHead{ + head: head, + mint: mint, + maxt: maxt, + } +} + +func (h *RangeHead) Index() (IndexReader, error) { + return h.head.indexRange(h.mint, h.maxt), nil +} + +func (h *RangeHead) Chunks() (ChunkReader, error) { + return h.head.chunksRange(h.mint, h.maxt, h.head.iso.State(h.mint, h.maxt)) +} + +func (h *RangeHead) Tombstones() (tombstones.Reader, error) { + return h.head.tombstones, nil +} + +func (h *RangeHead) MinTime() int64 { + return h.mint +} + +// MaxTime returns the max time of actual data fetch-able from the head. +// This controls the chunks time range which is closed [b.MinTime, b.MaxTime]. +func (h *RangeHead) MaxTime() int64 { + return h.maxt +} + +// BlockMaxTime returns the max time of the potential block created from this head. +// It's different to MaxTime as we need to add +1 millisecond to block maxt because block +// intervals are half-open: [b.MinTime, b.MaxTime). Block intervals are always +1 than the total samples it includes. +func (h *RangeHead) BlockMaxTime() int64 { + return h.MaxTime() + 1 +} + +func (h *RangeHead) NumSeries() uint64 { + return h.head.NumSeries() +} + +func (h *RangeHead) Meta() BlockMeta { + return BlockMeta{ + MinTime: h.MinTime(), + MaxTime: h.MaxTime(), + ULID: h.head.Meta().ULID, + Stats: BlockStats{ + NumSeries: h.NumSeries(), + }, + } +} + +// String returns an human readable representation of the range head. It's important to +// keep this function in order to avoid the struct dump when the head is stringified in +// errors or logs. +func (h *RangeHead) String() string { + return fmt.Sprintf("range head (mint: %d, maxt: %d)", h.MinTime(), h.MaxTime()) +} + +// Delete all samples in the range of [mint, maxt] for series that satisfy the given +// label matchers. +func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error { + // Do not delete anything beyond the currently valid range. + mint, maxt = clampInterval(mint, maxt, h.MinTime(), h.MaxTime()) + + ir := h.indexRange(mint, maxt) + + p, err := PostingsForMatchers(ir, ms...) + if err != nil { + return errors.Wrap(err, "select series") + } + + var stones []tombstones.Stone + for p.Next() { + series := h.series.getByID(chunks.HeadSeriesRef(p.At())) + + series.RLock() + t0, t1 := series.minTime(), series.maxTime() + series.RUnlock() + if t0 == math.MinInt64 || t1 == math.MinInt64 { + continue + } + // Delete only until the current values and not beyond. + t0, t1 = clampInterval(mint, maxt, t0, t1) + stones = append(stones, tombstones.Stone{Ref: p.At(), Intervals: tombstones.Intervals{{Mint: t0, Maxt: t1}}}) + } + if p.Err() != nil { + return p.Err() + } + if h.wal != nil { + var enc record.Encoder + if err := h.wal.Log(enc.Tombstones(stones, nil)); err != nil { + return err + } + } + for _, s := range stones { + h.tombstones.AddInterval(storage.SeriesRef(s.Ref), s.Intervals[0]) + } + + return nil +} + +// gc removes data before the minimum timestamp from the head. +// It returns the actual min times of the chunks present in the Head. +func (h *Head) gc() int64 { + // Only data strictly lower than this timestamp must be deleted. + mint := h.MinTime() + + // Drop old chunks and remember series IDs and hashes if they can be + // deleted entirely. + deleted, chunksRemoved, actualMint := h.series.gc(mint) + seriesRemoved := len(deleted) + + h.metrics.seriesRemoved.Add(float64(seriesRemoved)) + h.metrics.chunksRemoved.Add(float64(chunksRemoved)) + h.metrics.chunks.Sub(float64(chunksRemoved)) + h.numSeries.Sub(uint64(seriesRemoved)) + + // Remove deleted series IDs from the postings lists. + h.postings.Delete(deleted) + + // Remove tombstones referring to the deleted series. + h.tombstones.DeleteTombstones(deleted) + h.tombstones.TruncateBefore(mint) + + if h.wal != nil { + _, last, _ := wal.Segments(h.wal.Dir()) + h.deletedMtx.Lock() + // Keep series records until we're past segment 'last' + // because the WAL will still have samples records with + // this ref ID. If we didn't keep these series records then + // on start up when we replay the WAL, or any other code + // that reads the WAL, wouldn't be able to use those + // samples since we would have no labels for that ref ID. + for ref := range deleted { + h.deleted[chunks.HeadSeriesRef(ref)] = last + } + h.deletedMtx.Unlock() + } + + return actualMint +} + +// Tombstones returns a new reader over the head's tombstones +func (h *Head) Tombstones() (tombstones.Reader, error) { + return h.tombstones, nil +} + +// NumSeries returns the number of active series in the head. +func (h *Head) NumSeries() uint64 { + return h.numSeries.Load() +} + +// Meta returns meta information about the head. +// The head is dynamic so will return dynamic results. +func (h *Head) Meta() BlockMeta { + var id [16]byte + copy(id[:], "______head______") + return BlockMeta{ + MinTime: h.MinTime(), + MaxTime: h.MaxTime(), + ULID: ulid.ULID(id), + Stats: BlockStats{ + NumSeries: h.NumSeries(), + }, + } +} + +// MinTime returns the lowest time bound on visible data in the head. +func (h *Head) MinTime() int64 { + return h.minTime.Load() +} + +// MaxTime returns the highest timestamp seen in data of the head. +func (h *Head) MaxTime() int64 { + return h.maxTime.Load() +} + +// compactable returns whether the head has a compactable range. +// The head has a compactable range when the head time range is 1.5 times the chunk range. +// The 0.5 acts as a buffer of the appendable window. +func (h *Head) compactable() bool { + return h.MaxTime()-h.MinTime() > h.chunkRange.Load()/2*3 +} + +// Close flushes the WAL and closes the head. +// It also takes a snapshot of in-memory chunks if enabled. +func (h *Head) Close() error { + h.closedMtx.Lock() + defer h.closedMtx.Unlock() + h.closed = true + errs := tsdb_errors.NewMulti(h.chunkDiskMapper.Close()) + if h.wal != nil { + errs.Add(h.wal.Close()) + } + if errs.Err() == nil && h.opts.EnableMemorySnapshotOnShutdown { + errs.Add(h.performChunkSnapshot()) + } + return errs.Err() +} + +// String returns an human readable representation of the TSDB head. It's important to +// keep this function in order to avoid the struct dump when the head is stringified in +// errors or logs. +func (h *Head) String() string { + return "head" +} + +func (h *Head) getOrCreate(hash uint64, lset labels.Labels) (*memSeries, bool, error) { + // Just using `getOrCreateWithID` below would be semantically sufficient, but we'd create + // a new series on every sample inserted via Add(), which causes allocations + // and makes our series IDs rather random and harder to compress in postings. + s := h.series.getByHash(hash, lset) + if s != nil { + return s, false, nil + } + + // Optimistically assume that we are the first one to create the series. + id := chunks.HeadSeriesRef(h.lastSeriesID.Inc()) + + return h.getOrCreateWithID(id, hash, lset) +} + +func (h *Head) getOrCreateWithID(id chunks.HeadSeriesRef, hash uint64, lset labels.Labels) (*memSeries, bool, error) { + s, created, err := h.series.getOrSet(hash, lset, func() *memSeries { + return newMemSeries(lset, id, h.chunkRange.Load(), &h.memChunkPool, h.opts.IsolationDisabled) + }) + if err != nil { + return nil, false, err + } + if !created { + return s, false, nil + } + + h.metrics.seriesCreated.Inc() + h.numSeries.Inc() + + h.postings.Add(storage.SeriesRef(id), lset) + return s, true, nil +} + +// seriesHashmap is a simple hashmap for memSeries by their label set. It is built +// on top of a regular hashmap and holds a slice of series to resolve hash collisions. +// Its methods require the hash to be submitted with it to avoid re-computations throughout +// the code. +type seriesHashmap map[uint64][]*memSeries + +func (m seriesHashmap) get(hash uint64, lset labels.Labels) *memSeries { + for _, s := range m[hash] { + if labels.Equal(s.lset, lset) { + return s + } + } + return nil +} + +func (m seriesHashmap) set(hash uint64, s *memSeries) { + l := m[hash] + for i, prev := range l { + if labels.Equal(prev.lset, s.lset) { + l[i] = s + return + } + } + m[hash] = append(l, s) +} + +func (m seriesHashmap) del(hash uint64, lset labels.Labels) { + var rem []*memSeries + for _, s := range m[hash] { + if !labels.Equal(s.lset, lset) { + rem = append(rem, s) + } + } + if len(rem) == 0 { + delete(m, hash) + } else { + m[hash] = rem + } +} + +const ( + // DefaultStripeSize is the default number of entries to allocate in the stripeSeries hash map. + DefaultStripeSize = 1 << 14 +) + +// stripeSeries holds series by HeadSeriesRef ("ID") and also by hash of their labels. +// ID-based lookups via (getByID()) are preferred over getByHash() for performance reasons. +// It locks modulo ranges of IDs and hashes to reduce lock contention. +// The locks are padded to not be on the same cache line. Filling the padded space +// with the maps was profiled to be slower – likely due to the additional pointer +// dereferences. +type stripeSeries struct { + size int + series []map[chunks.HeadSeriesRef]*memSeries // Sharded by ref. A series ref is the value of `size` when the series was being newly added. + hashes []seriesHashmap // Sharded by label hash. + locks []stripeLock // Sharded by ref for series access, by label hash for hashes access. + seriesLifecycleCallback SeriesLifecycleCallback +} + +type stripeLock struct { + sync.RWMutex + // Padding to avoid multiple locks being on the same cache line. + _ [40]byte +} + +func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *stripeSeries { + s := &stripeSeries{ + size: stripeSize, + series: make([]map[chunks.HeadSeriesRef]*memSeries, stripeSize), + hashes: make([]seriesHashmap, stripeSize), + locks: make([]stripeLock, stripeSize), + seriesLifecycleCallback: seriesCallback, + } + + for i := range s.series { + s.series[i] = map[chunks.HeadSeriesRef]*memSeries{} + } + for i := range s.hashes { + s.hashes[i] = seriesHashmap{} + } + return s +} + +// gc garbage collects old chunks that are strictly before mint and removes +// series entirely that have no chunks left. +// note: returning map[chunks.HeadSeriesRef]struct{} would be more accurate, +// but the returned map goes into postings.Delete() which expects a map[storage.SeriesRef]struct +// and there's no easy way to cast maps. +func (s *stripeSeries) gc(mint int64) (map[storage.SeriesRef]struct{}, int, int64) { + var ( + deleted = map[storage.SeriesRef]struct{}{} + deletedForCallback = []labels.Labels{} + rmChunks = 0 + actualMint int64 = math.MaxInt64 + ) + // Run through all series and truncate old chunks. Mark those with no + // chunks left as deleted and store their ID. + for i := 0; i < s.size; i++ { + s.locks[i].Lock() + + for hash, all := range s.hashes[i] { + for _, series := range all { + series.Lock() + rmChunks += series.truncateChunksBefore(mint) + + if len(series.mmappedChunks) > 0 || series.headChunk != nil || series.pendingCommit { + seriesMint := series.minTime() + if seriesMint < actualMint { + actualMint = seriesMint + } + series.Unlock() + continue + } + + // The series is gone entirely. We need to keep the series lock + // and make sure we have acquired the stripe locks for hash and ID of the + // series alike. + // If we don't hold them all, there's a very small chance that a series receives + // samples again while we are half-way into deleting it. + j := int(series.ref) & (s.size - 1) + + if i != j { + s.locks[j].Lock() + } + + deleted[storage.SeriesRef(series.ref)] = struct{}{} + s.hashes[i].del(hash, series.lset) + delete(s.series[j], series.ref) + deletedForCallback = append(deletedForCallback, series.lset) + + if i != j { + s.locks[j].Unlock() + } + + series.Unlock() + } + } + + s.locks[i].Unlock() + + s.seriesLifecycleCallback.PostDeletion(deletedForCallback...) + deletedForCallback = deletedForCallback[:0] + } + + if actualMint == math.MaxInt64 { + actualMint = mint + } + + return deleted, rmChunks, actualMint +} + +func (s *stripeSeries) getByID(id chunks.HeadSeriesRef) *memSeries { + i := uint64(id) & uint64(s.size-1) + + s.locks[i].RLock() + series := s.series[i][id] + s.locks[i].RUnlock() + + return series +} + +func (s *stripeSeries) getByHash(hash uint64, lset labels.Labels) *memSeries { + i := hash & uint64(s.size-1) + + s.locks[i].RLock() + series := s.hashes[i].get(hash, lset) + s.locks[i].RUnlock() + + return series +} + +func (s *stripeSeries) getOrSet(hash uint64, lset labels.Labels, createSeries func() *memSeries) (*memSeries, bool, error) { + // PreCreation is called here to avoid calling it inside the lock. + // It is not necessary to call it just before creating a series, + // rather it gives a 'hint' whether to create a series or not. + preCreationErr := s.seriesLifecycleCallback.PreCreation(lset) + + // Create the series, unless the PreCreation() callback as failed. + // If failed, we'll not allow to create a new series anyway. + var series *memSeries + if preCreationErr == nil { + series = createSeries() + } + + i := hash & uint64(s.size-1) + s.locks[i].Lock() + + if prev := s.hashes[i].get(hash, lset); prev != nil { + s.locks[i].Unlock() + return prev, false, nil + } + if preCreationErr == nil { + s.hashes[i].set(hash, series) + } + s.locks[i].Unlock() + + if preCreationErr != nil { + // The callback prevented creation of series. + return nil, false, preCreationErr + } + // Setting the series in the s.hashes marks the creation of series + // as any further calls to this methods would return that series. + s.seriesLifecycleCallback.PostCreation(series.lset) + + i = uint64(series.ref) & uint64(s.size-1) + + s.locks[i].Lock() + s.series[i][series.ref] = series + s.locks[i].Unlock() + + return series, true, nil +} + +type sample struct { + t int64 + v float64 +} + +func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v} } +func (s sample) T() int64 { return s.t } +func (s sample) V() float64 { return s.v } + +// memSeries is the in-memory representation of a series. None of its methods +// are goroutine safe and it is the caller's responsibility to lock it. +type memSeries struct { + sync.RWMutex + + ref chunks.HeadSeriesRef + lset labels.Labels + + // Immutable chunks on disk that have not yet gone into a block, in order of ascending time stamps. + // When compaction runs, chunks get moved into a block and all pointers are shifted like so: + // + // /------- let's say these 2 chunks get stored into a block + // | | + // before compaction: mmappedChunks=[p5,p6,p7,p8,p9] firstChunkID=5 + // after compaction: mmappedChunks=[p7,p8,p9] firstChunkID=7 + // + // pN is the pointer to the mmappedChunk referered to by HeadChunkID=N + mmappedChunks []*mmappedChunk + + mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay. + headChunk *memChunk // Most recent chunk in memory that's still being built. + chunkRange int64 + firstChunkID chunks.HeadChunkID // HeadChunkID for mmappedChunks[0] + + nextAt int64 // Timestamp at which to cut the next chunk. + + // We keep the last 4 samples here (in addition to appending them to the chunk) so we don't need coordination between appender and querier. + // Even the most compact encoding of a sample takes 2 bits, so the last byte is not contended. + sampleBuf [4]sample + + pendingCommit bool // Whether there are samples waiting to be committed to this series. + + // Current appender for the head chunk. Set when a new head chunk is cut. + // It is nil only if headChunk is nil. E.g. if there was an appender that created a new series, but rolled back the commit + // (the first sample would create a headChunk, hence appender, but rollback skipped it while the Append() call would create a series). + app chunkenc.Appender + + memChunkPool *sync.Pool + + // txs is nil if isolation is disabled. + txs *txRing +} + +func newMemSeries(lset labels.Labels, id chunks.HeadSeriesRef, chunkRange int64, memChunkPool *sync.Pool, isolationDisabled bool) *memSeries { + s := &memSeries{ + lset: lset, + ref: id, + chunkRange: chunkRange, + nextAt: math.MinInt64, + memChunkPool: memChunkPool, + } + if !isolationDisabled { + s.txs = newTxRing(4) + } + return s +} + +func (s *memSeries) minTime() int64 { + if len(s.mmappedChunks) > 0 { + return s.mmappedChunks[0].minTime + } + if s.headChunk != nil { + return s.headChunk.minTime + } + return math.MinInt64 +} + +func (s *memSeries) maxTime() int64 { + c := s.head() + if c != nil { + return c.maxTime + } + if len(s.mmappedChunks) > 0 { + return s.mmappedChunks[len(s.mmappedChunks)-1].maxTime + } + return math.MinInt64 +} + +// truncateChunksBefore removes all chunks from the series that +// have no timestamp at or after mint. +// Chunk IDs remain unchanged. +func (s *memSeries) truncateChunksBefore(mint int64) (removed int) { + if s.headChunk != nil && s.headChunk.maxTime < mint { + // If head chunk is truncated, we can truncate all mmapped chunks. + removed = 1 + len(s.mmappedChunks) + s.firstChunkID += chunks.HeadChunkID(removed) + s.headChunk = nil + s.mmappedChunks = nil + return removed + } + if len(s.mmappedChunks) > 0 { + for i, c := range s.mmappedChunks { + if c.maxTime >= mint { + break + } + removed = i + 1 + } + s.mmappedChunks = append(s.mmappedChunks[:0], s.mmappedChunks[removed:]...) + s.firstChunkID += chunks.HeadChunkID(removed) + } + return removed +} + +// cleanupAppendIDsBelow cleans up older appendIDs. Has to be called after +// acquiring lock. +func (s *memSeries) cleanupAppendIDsBelow(bound uint64) { + if s.txs != nil { + s.txs.cleanupAppendIDsBelow(bound) + } +} + +func (s *memSeries) head() *memChunk { + return s.headChunk +} + +type memChunk struct { + chunk chunkenc.Chunk + minTime, maxTime int64 +} + +// OverlapsClosedInterval returns true if the chunk overlaps [mint, maxt]. +func (mc *memChunk) OverlapsClosedInterval(mint, maxt int64) bool { + return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt) +} + +func overlapsClosedInterval(mint1, maxt1, mint2, maxt2 int64) bool { + return mint1 <= maxt2 && mint2 <= maxt1 +} + +// mappedChunks describes a head chunk on disk that has been mmapped +type mmappedChunk struct { + ref chunks.ChunkDiskMapperRef + numSamples uint16 + minTime, maxTime int64 +} + +// Returns true if the chunk overlaps [mint, maxt]. +func (mc *mmappedChunk) OverlapsClosedInterval(mint, maxt int64) bool { + return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt) +} + +type noopSeriesLifecycleCallback struct{} + +func (noopSeriesLifecycleCallback) PreCreation(labels.Labels) error { return nil } +func (noopSeriesLifecycleCallback) PostCreation(labels.Labels) {} +func (noopSeriesLifecycleCallback) PostDeletion(...labels.Labels) {} + +func (h *Head) Size() int64 { + var walSize int64 + if h.wal != nil { + walSize, _ = h.wal.Size() + } + cdmSize, _ := h.chunkDiskMapper.Size() + return walSize + cdmSize +} + +func (h *RangeHead) Size() int64 { + return h.head.Size() +} + +func (h *Head) startWALReplayStatus(startFrom, last int) { + h.stats.WALReplayStatus.Lock() + defer h.stats.WALReplayStatus.Unlock() + + h.stats.WALReplayStatus.Min = startFrom + h.stats.WALReplayStatus.Max = last + h.stats.WALReplayStatus.Current = startFrom +} + +func (h *Head) updateWALReplayStatusRead(current int) { + h.stats.WALReplayStatus.Lock() + defer h.stats.WALReplayStatus.Unlock() + + h.stats.WALReplayStatus.Current = current +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_append.go b/vendor/github.com/prometheus/prometheus/tsdb/head_append.go new file mode 100644 index 00000000000..8f1e9bc54a6 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_append.go @@ -0,0 +1,623 @@ +// Copyright 2021 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "fmt" + "math" + + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/record" +) + +// initAppender is a helper to initialize the time bounds of the head +// upon the first sample it receives. +type initAppender struct { + app storage.Appender + head *Head +} + +var _ storage.GetRef = &initAppender{} + +func (a *initAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + if a.app != nil { + return a.app.Append(ref, lset, t, v) + } + + a.head.initTime(t) + a.app = a.head.appender() + return a.app.Append(ref, lset, t, v) +} + +func (a *initAppender) AppendExemplar(ref storage.SeriesRef, l labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { + // Check if exemplar storage is enabled. + if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { + return 0, nil + } + + if a.app != nil { + return a.app.AppendExemplar(ref, l, e) + } + // We should never reach here given we would call Append before AppendExemplar + // and we probably want to always base head/WAL min time on sample times. + a.head.initTime(e.Ts) + a.app = a.head.appender() + + return a.app.AppendExemplar(ref, l, e) +} + +// initTime initializes a head with the first timestamp. This only needs to be called +// for a completely fresh head with an empty WAL. +func (h *Head) initTime(t int64) { + if !h.minTime.CAS(math.MaxInt64, t) { + return + } + // Ensure that max time is initialized to at least the min time we just set. + // Concurrent appenders may already have set it to a higher value. + h.maxTime.CAS(math.MinInt64, t) +} + +func (a *initAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { + if g, ok := a.app.(storage.GetRef); ok { + return g.GetRef(lset) + } + return 0, nil +} + +func (a *initAppender) Commit() error { + if a.app == nil { + a.head.metrics.activeAppenders.Dec() + return nil + } + return a.app.Commit() +} + +func (a *initAppender) Rollback() error { + if a.app == nil { + a.head.metrics.activeAppenders.Dec() + return nil + } + return a.app.Rollback() +} + +// Appender returns a new Appender on the database. +func (h *Head) Appender(_ context.Context) storage.Appender { + h.metrics.activeAppenders.Inc() + + // The head cache might not have a starting point yet. The init appender + // picks up the first appended timestamp as the base. + if h.MinTime() == math.MaxInt64 { + return &initAppender{ + head: h, + } + } + return h.appender() +} + +func (h *Head) appender() *headAppender { + appendID, cleanupAppendIDsBelow := h.iso.newAppendID() // Every appender gets an ID that is cleared upon commit/rollback. + + // Allocate the exemplars buffer only if exemplars are enabled. + var exemplarsBuf []exemplarWithSeriesRef + if h.opts.EnableExemplarStorage { + exemplarsBuf = h.getExemplarBuffer() + } + + return &headAppender{ + head: h, + minValidTime: h.appendableMinValidTime(), + mint: math.MaxInt64, + maxt: math.MinInt64, + samples: h.getAppendBuffer(), + sampleSeries: h.getSeriesBuffer(), + exemplars: exemplarsBuf, + appendID: appendID, + cleanupAppendIDsBelow: cleanupAppendIDsBelow, + } +} + +// appendableMinValidTime returns the minimum valid timestamp for appends, +// such that samples stay ahead of prior blocks and the head compaction window. +func (h *Head) appendableMinValidTime() int64 { + // This boundary ensures that no samples will be added to the compaction window. + // This allows race-free, concurrent appending and compaction. + cwEnd := h.MaxTime() - h.chunkRange.Load()/2 + + // This boundary ensures that we avoid overlapping timeframes from one block to the next. + // While not necessary for correctness, it means we're not required to use vertical compaction. + minValid := h.minValidTime.Load() + + return max(cwEnd, minValid) +} + +// AppendableMinValidTime returns the minimum valid time for samples to be appended to the Head. +// Returns false if Head hasn't been initialized yet and the minimum time isn't known yet. +func (h *Head) AppendableMinValidTime() (int64, bool) { + if h.MinTime() == math.MaxInt64 { + return 0, false + } + + return h.appendableMinValidTime(), true +} + +func max(a, b int64) int64 { + if a > b { + return a + } + return b +} + +func (h *Head) getAppendBuffer() []record.RefSample { + b := h.appendPool.Get() + if b == nil { + return make([]record.RefSample, 0, 512) + } + return b.([]record.RefSample) +} + +func (h *Head) putAppendBuffer(b []record.RefSample) { + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + h.appendPool.Put(b[:0]) +} + +func (h *Head) getExemplarBuffer() []exemplarWithSeriesRef { + b := h.exemplarsPool.Get() + if b == nil { + return make([]exemplarWithSeriesRef, 0, 512) + } + return b.([]exemplarWithSeriesRef) +} + +func (h *Head) putExemplarBuffer(b []exemplarWithSeriesRef) { + if b == nil { + return + } + + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + h.exemplarsPool.Put(b[:0]) +} + +func (h *Head) getSeriesBuffer() []*memSeries { + b := h.seriesPool.Get() + if b == nil { + return make([]*memSeries, 0, 512) + } + return b.([]*memSeries) +} + +func (h *Head) putSeriesBuffer(b []*memSeries) { + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + h.seriesPool.Put(b[:0]) +} + +func (h *Head) getBytesBuffer() []byte { + b := h.bytesPool.Get() + if b == nil { + return make([]byte, 0, 1024) + } + return b.([]byte) +} + +func (h *Head) putBytesBuffer(b []byte) { + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + h.bytesPool.Put(b[:0]) +} + +type exemplarWithSeriesRef struct { + ref storage.SeriesRef + exemplar exemplar.Exemplar +} + +type headAppender struct { + head *Head + minValidTime int64 // No samples below this timestamp are allowed. + mint, maxt int64 + + series []record.RefSeries // New series held by this appender. + samples []record.RefSample // New samples held by this appender. + exemplars []exemplarWithSeriesRef // New exemplars held by this appender. + sampleSeries []*memSeries // Series corresponding to the samples held by this appender (using corresponding slice indices - same series may appear more than once). + + appendID, cleanupAppendIDsBelow uint64 + closed bool +} + +func (a *headAppender) Append(ref storage.SeriesRef, lset labels.Labels, t int64, v float64) (storage.SeriesRef, error) { + if t < a.minValidTime { + a.head.metrics.outOfBoundSamples.Inc() + return 0, storage.ErrOutOfBounds + } + + s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) + if s == nil { + // Ensure no empty labels have gotten through. + lset = lset.WithoutEmpty() + if len(lset) == 0 { + return 0, errors.Wrap(ErrInvalidSample, "empty labelset") + } + + if l, dup := lset.HasDuplicateLabelNames(); dup { + return 0, errors.Wrap(ErrInvalidSample, fmt.Sprintf(`label name "%s" is not unique`, l)) + } + + var created bool + var err error + s, created, err = a.head.getOrCreate(lset.Hash(), lset) + if err != nil { + return 0, err + } + if created { + a.series = append(a.series, record.RefSeries{ + Ref: s.ref, + Labels: lset, + }) + } + } + + s.Lock() + if err := s.appendable(t, v); err != nil { + s.Unlock() + if err == storage.ErrOutOfOrderSample { + a.head.metrics.outOfOrderSamples.Inc() + } + return 0, err + } + s.pendingCommit = true + s.Unlock() + + if t < a.mint { + a.mint = t + } + if t > a.maxt { + a.maxt = t + } + + a.samples = append(a.samples, record.RefSample{ + Ref: s.ref, + T: t, + V: v, + }) + a.sampleSeries = append(a.sampleSeries, s) + return storage.SeriesRef(s.ref), nil +} + +// appendable checks whether the given sample is valid for appending to the series. +func (s *memSeries) appendable(t int64, v float64) error { + c := s.head() + if c == nil { + return nil + } + + if t > c.maxTime { + return nil + } + if t < c.maxTime { + return storage.ErrOutOfOrderSample + } + // We are allowing exact duplicates as we can encounter them in valid cases + // like federation and erroring out at that time would be extremely noisy. + if math.Float64bits(s.sampleBuf[3].v) != math.Float64bits(v) { + return storage.ErrDuplicateSampleForTimestamp + } + return nil +} + +// AppendExemplar for headAppender assumes the series ref already exists, and so it doesn't +// use getOrCreate or make any of the lset sanity checks that Append does. +func (a *headAppender) AppendExemplar(ref storage.SeriesRef, lset labels.Labels, e exemplar.Exemplar) (storage.SeriesRef, error) { + // Check if exemplar storage is enabled. + if !a.head.opts.EnableExemplarStorage || a.head.opts.MaxExemplars.Load() <= 0 { + return 0, nil + } + + // Get Series + s := a.head.series.getByID(chunks.HeadSeriesRef(ref)) + if s == nil { + s = a.head.series.getByHash(lset.Hash(), lset) + if s != nil { + ref = storage.SeriesRef(s.ref) + } + } + if s == nil { + return 0, fmt.Errorf("unknown HeadSeriesRef when trying to add exemplar: %d", ref) + } + + // Ensure no empty labels have gotten through. + e.Labels = e.Labels.WithoutEmpty() + + err := a.head.exemplars.ValidateExemplar(s.lset, e) + if err != nil { + if err == storage.ErrDuplicateExemplar || err == storage.ErrExemplarsDisabled { + // Duplicate, don't return an error but don't accept the exemplar. + return 0, nil + } + return 0, err + } + + a.exemplars = append(a.exemplars, exemplarWithSeriesRef{ref, e}) + + return storage.SeriesRef(s.ref), nil +} + +var _ storage.GetRef = &headAppender{} + +func (a *headAppender) GetRef(lset labels.Labels) (storage.SeriesRef, labels.Labels) { + s := a.head.series.getByHash(lset.Hash(), lset) + if s == nil { + return 0, nil + } + // returned labels must be suitable to pass to Append() + return storage.SeriesRef(s.ref), s.lset +} + +// log writes all headAppender's data to the WAL. +func (a *headAppender) log() error { + if a.head.wal == nil { + return nil + } + + buf := a.head.getBytesBuffer() + defer func() { a.head.putBytesBuffer(buf) }() + + var rec []byte + var enc record.Encoder + + if len(a.series) > 0 { + rec = enc.Series(a.series, buf) + buf = rec[:0] + + if err := a.head.wal.Log(rec); err != nil { + return errors.Wrap(err, "log series") + } + } + if len(a.samples) > 0 { + rec = enc.Samples(a.samples, buf) + buf = rec[:0] + + if err := a.head.wal.Log(rec); err != nil { + return errors.Wrap(err, "log samples") + } + } + if len(a.exemplars) > 0 { + rec = enc.Exemplars(exemplarsForEncoding(a.exemplars), buf) + buf = rec[:0] + + if err := a.head.wal.Log(rec); err != nil { + return errors.Wrap(err, "log exemplars") + } + } + return nil +} + +func exemplarsForEncoding(es []exemplarWithSeriesRef) []record.RefExemplar { + ret := make([]record.RefExemplar, 0, len(es)) + for _, e := range es { + ret = append(ret, record.RefExemplar{ + Ref: chunks.HeadSeriesRef(e.ref), + T: e.exemplar.Ts, + V: e.exemplar.Value, + Labels: e.exemplar.Labels, + }) + } + return ret +} + +// Commit writes to the WAL and adds the data to the Head. +func (a *headAppender) Commit() (err error) { + if a.closed { + return ErrAppenderClosed + } + defer func() { a.closed = true }() + + if err := a.log(); err != nil { + _ = a.Rollback() // Most likely the same error will happen again. + return errors.Wrap(err, "write to WAL") + } + + // No errors logging to WAL, so pass the exemplars along to the in memory storage. + for _, e := range a.exemplars { + s := a.head.series.getByID(chunks.HeadSeriesRef(e.ref)) + // We don't instrument exemplar appends here, all is instrumented by storage. + if err := a.head.exemplars.AddExemplar(s.lset, e.exemplar); err != nil { + if err == storage.ErrOutOfOrderExemplar { + continue + } + level.Debug(a.head.logger).Log("msg", "Unknown error while adding exemplar", "err", err) + } + } + + defer a.head.metrics.activeAppenders.Dec() + defer a.head.putAppendBuffer(a.samples) + defer a.head.putSeriesBuffer(a.sampleSeries) + defer a.head.putExemplarBuffer(a.exemplars) + defer a.head.iso.closeAppend(a.appendID) + + total := len(a.samples) + var series *memSeries + for i, s := range a.samples { + series = a.sampleSeries[i] + series.Lock() + ok, chunkCreated := series.append(s.T, s.V, a.appendID, a.head.chunkDiskMapper) + series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) + series.pendingCommit = false + series.Unlock() + + if !ok { + total-- + a.head.metrics.outOfOrderSamples.Inc() + } + if chunkCreated { + a.head.metrics.chunks.Inc() + a.head.metrics.chunksCreated.Inc() + } + } + + a.head.metrics.samplesAppended.Add(float64(total)) + a.head.updateMinMaxTime(a.mint, a.maxt) + + return nil +} + +// append adds the sample (t, v) to the series. The caller also has to provide +// the appendID for isolation. (The appendID can be zero, which results in no +// isolation for this append.) +// It is unsafe to call this concurrently with s.iterator(...) without holding the series lock. +func (s *memSeries) append(t int64, v float64, appendID uint64, chunkDiskMapper *chunks.ChunkDiskMapper) (sampleInOrder, chunkCreated bool) { + // Based on Gorilla white papers this offers near-optimal compression ratio + // so anything bigger that this has diminishing returns and increases + // the time range within which we have to decompress all samples. + const samplesPerChunk = 120 + + c := s.head() + + if c == nil { + if len(s.mmappedChunks) > 0 && s.mmappedChunks[len(s.mmappedChunks)-1].maxTime >= t { + // Out of order sample. Sample timestamp is already in the mmapped chunks, so ignore it. + return false, false + } + // There is no chunk in this series yet, create the first chunk for the sample. + c = s.cutNewHeadChunk(t, chunkDiskMapper) + chunkCreated = true + } + + // Out of order sample. + if c.maxTime >= t { + return false, chunkCreated + } + + numSamples := c.chunk.NumSamples() + if numSamples == 0 { + // It could be the new chunk created after reading the chunk snapshot, + // hence we fix the minTime of the chunk here. + c.minTime = t + s.nextAt = rangeForTimestamp(c.minTime, s.chunkRange) + } + + // If we reach 25% of a chunk's desired sample count, predict an end time + // for this chunk that will try to make samples equally distributed within + // the remaining chunks in the current chunk range. + // At latest it must happen at the timestamp set when the chunk was cut. + if numSamples == samplesPerChunk/4 { + s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, s.nextAt) + } + if t >= s.nextAt { + c = s.cutNewHeadChunk(t, chunkDiskMapper) + chunkCreated = true + } + s.app.Append(t, v) + + c.maxTime = t + + s.sampleBuf[0] = s.sampleBuf[1] + s.sampleBuf[1] = s.sampleBuf[2] + s.sampleBuf[2] = s.sampleBuf[3] + s.sampleBuf[3] = sample{t: t, v: v} + + if appendID > 0 && s.txs != nil { + s.txs.add(appendID) + } + + return true, chunkCreated +} + +// computeChunkEndTime estimates the end timestamp based the beginning of a +// chunk, its current timestamp and the upper bound up to which we insert data. +// It assumes that the time range is 1/4 full. +// Assuming that the samples will keep arriving at the same rate, it will make the +// remaining n chunks within this chunk range (before max) equally sized. +func computeChunkEndTime(start, cur, max int64) int64 { + n := (max - start) / ((cur - start + 1) * 4) + if n <= 1 { + return max + } + return start + (max-start)/n +} + +func (s *memSeries) cutNewHeadChunk(mint int64, chunkDiskMapper *chunks.ChunkDiskMapper) *memChunk { + s.mmapCurrentHeadChunk(chunkDiskMapper) + + s.headChunk = &memChunk{ + chunk: chunkenc.NewXORChunk(), + minTime: mint, + maxTime: math.MinInt64, + } + + // Set upper bound on when the next chunk must be started. An earlier timestamp + // may be chosen dynamically at a later point. + s.nextAt = rangeForTimestamp(mint, s.chunkRange) + + app, err := s.headChunk.chunk.Appender() + if err != nil { + panic(err) + } + s.app = app + return s.headChunk +} + +func (s *memSeries) mmapCurrentHeadChunk(chunkDiskMapper *chunks.ChunkDiskMapper) { + if s.headChunk == nil { + // There is no head chunk, so nothing to m-map here. + return + } + + chunkRef := chunkDiskMapper.WriteChunk(s.ref, s.headChunk.minTime, s.headChunk.maxTime, s.headChunk.chunk, handleChunkWriteError) + s.mmappedChunks = append(s.mmappedChunks, &mmappedChunk{ + ref: chunkRef, + numSamples: uint16(s.headChunk.chunk.NumSamples()), + minTime: s.headChunk.minTime, + maxTime: s.headChunk.maxTime, + }) +} + +func handleChunkWriteError(err error) { + if err != nil && err != chunks.ErrChunkDiskMapperClosed { + panic(err) + } +} + +// Rollback removes the samples and exemplars from headAppender and writes any series to WAL. +func (a *headAppender) Rollback() (err error) { + if a.closed { + return ErrAppenderClosed + } + defer func() { a.closed = true }() + defer a.head.metrics.activeAppenders.Dec() + defer a.head.iso.closeAppend(a.appendID) + defer a.head.putSeriesBuffer(a.sampleSeries) + + var series *memSeries + for i := range a.samples { + series = a.sampleSeries[i] + series.Lock() + series.cleanupAppendIDsBelow(a.cleanupAppendIDsBelow) + series.pendingCommit = false + series.Unlock() + } + a.head.putAppendBuffer(a.samples) + a.head.putExemplarBuffer(a.exemplars) + a.samples = nil + a.exemplars = nil + + // Series are created in the head memory regardless of rollback. Thus we have + // to log them to the WAL in any case. + return a.log() +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_read.go b/vendor/github.com/prometheus/prometheus/tsdb/head_read.go new file mode 100644 index 00000000000..cd19c0ef58e --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_read.go @@ -0,0 +1,509 @@ +// Copyright 2021 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "math" + "sort" + + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" +) + +func (h *Head) ExemplarQuerier(ctx context.Context) (storage.ExemplarQuerier, error) { + return h.exemplars.ExemplarQuerier(ctx) +} + +// Index returns an IndexReader against the block. +func (h *Head) Index() (IndexReader, error) { + return h.indexRange(math.MinInt64, math.MaxInt64), nil +} + +func (h *Head) indexRange(mint, maxt int64) *headIndexReader { + if hmin := h.MinTime(); hmin > mint { + mint = hmin + } + return &headIndexReader{head: h, mint: mint, maxt: maxt} +} + +type headIndexReader struct { + head *Head + mint, maxt int64 +} + +func (h *headIndexReader) Close() error { + return nil +} + +func (h *headIndexReader) Symbols() index.StringIter { + return h.head.postings.Symbols() +} + +// SortedLabelValues returns label values present in the head for the +// specific label name that are within the time range mint to maxt. +// If matchers are specified the returned result set is reduced +// to label values of metrics matching the matchers. +func (h *headIndexReader) SortedLabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + values, err := h.LabelValues(name, matchers...) + if err == nil { + sort.Strings(values) + } + return values, err +} + +// LabelValues returns label values present in the head for the +// specific label name that are within the time range mint to maxt. +// If matchers are specified the returned result set is reduced +// to label values of metrics matching the matchers. +func (h *headIndexReader) LabelValues(name string, matchers ...*labels.Matcher) ([]string, error) { + if h.maxt < h.head.MinTime() || h.mint > h.head.MaxTime() { + return []string{}, nil + } + + if len(matchers) == 0 { + return h.head.postings.LabelValues(name), nil + } + + return labelValuesWithMatchers(h, name, matchers...) +} + +// LabelNames returns all the unique label names present in the head +// that are within the time range mint to maxt. +func (h *headIndexReader) LabelNames(matchers ...*labels.Matcher) ([]string, error) { + if h.maxt < h.head.MinTime() || h.mint > h.head.MaxTime() { + return []string{}, nil + } + + if len(matchers) == 0 { + labelNames := h.head.postings.LabelNames() + sort.Strings(labelNames) + return labelNames, nil + } + + return labelNamesWithMatchers(h, matchers...) +} + +// Postings returns the postings list iterator for the label pairs. +func (h *headIndexReader) Postings(name string, values ...string) (index.Postings, error) { + switch len(values) { + case 0: + return index.EmptyPostings(), nil + case 1: + return h.head.postings.Get(name, values[0]), nil + default: + res := make([]index.Postings, 0, len(values)) + for _, value := range values { + res = append(res, h.head.postings.Get(name, value)) + } + return index.Merge(res...), nil + } +} + +func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings { + series := make([]*memSeries, 0, 128) + + // Fetch all the series only once. + for p.Next() { + s := h.head.series.getByID(chunks.HeadSeriesRef(p.At())) + if s == nil { + level.Debug(h.head.logger).Log("msg", "Looked up series not found") + } else { + series = append(series, s) + } + } + if err := p.Err(); err != nil { + return index.ErrPostings(errors.Wrap(err, "expand postings")) + } + + sort.Slice(series, func(i, j int) bool { + return labels.Compare(series[i].lset, series[j].lset) < 0 + }) + + // Convert back to list. + ep := make([]storage.SeriesRef, 0, len(series)) + for _, p := range series { + ep = append(ep, storage.SeriesRef(p.ref)) + } + return index.NewListPostings(ep) +} + +// Series returns the series for the given reference. +func (h *headIndexReader) Series(ref storage.SeriesRef, lbls *labels.Labels, chks *[]chunks.Meta) error { + s := h.head.series.getByID(chunks.HeadSeriesRef(ref)) + + if s == nil { + h.head.metrics.seriesNotFound.Inc() + return storage.ErrNotFound + } + *lbls = append((*lbls)[:0], s.lset...) + + s.Lock() + defer s.Unlock() + + *chks = (*chks)[:0] + + for i, c := range s.mmappedChunks { + // Do not expose chunks that are outside of the specified range. + if !c.OverlapsClosedInterval(h.mint, h.maxt) { + continue + } + *chks = append(*chks, chunks.Meta{ + MinTime: c.minTime, + MaxTime: c.maxTime, + Ref: chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.headChunkID(i))), + }) + } + if s.headChunk != nil && s.headChunk.OverlapsClosedInterval(h.mint, h.maxt) { + *chks = append(*chks, chunks.Meta{ + MinTime: s.headChunk.minTime, + MaxTime: math.MaxInt64, // Set the head chunks as open (being appended to). + Ref: chunks.ChunkRef(chunks.NewHeadChunkRef(s.ref, s.headChunkID(len(s.mmappedChunks)))), + }) + } + + return nil +} + +// headChunkID returns the HeadChunkID corresponding to .mmappedChunks[pos] +func (s *memSeries) headChunkID(pos int) chunks.HeadChunkID { + return chunks.HeadChunkID(pos) + s.firstChunkID +} + +// LabelValueFor returns label value for the given label name in the series referred to by ID. +func (h *headIndexReader) LabelValueFor(id storage.SeriesRef, label string) (string, error) { + memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id)) + if memSeries == nil { + return "", storage.ErrNotFound + } + + value := memSeries.lset.Get(label) + if value == "" { + return "", storage.ErrNotFound + } + + return value, nil +} + +// LabelNamesFor returns all the label names for the series referred to by IDs. +// The names returned are sorted. +func (h *headIndexReader) LabelNamesFor(ids ...storage.SeriesRef) ([]string, error) { + namesMap := make(map[string]struct{}) + for _, id := range ids { + memSeries := h.head.series.getByID(chunks.HeadSeriesRef(id)) + if memSeries == nil { + return nil, storage.ErrNotFound + } + for _, lbl := range memSeries.lset { + namesMap[lbl.Name] = struct{}{} + } + } + names := make([]string, 0, len(namesMap)) + for name := range namesMap { + names = append(names, name) + } + sort.Strings(names) + return names, nil +} + +// Chunks returns a ChunkReader against the block. +func (h *Head) Chunks() (ChunkReader, error) { + return h.chunksRange(math.MinInt64, math.MaxInt64, h.iso.State(math.MinInt64, math.MaxInt64)) +} + +func (h *Head) chunksRange(mint, maxt int64, is *isolationState) (*headChunkReader, error) { + h.closedMtx.Lock() + defer h.closedMtx.Unlock() + if h.closed { + return nil, errors.New("can't read from a closed head") + } + if hmin := h.MinTime(); hmin > mint { + mint = hmin + } + return &headChunkReader{ + head: h, + mint: mint, + maxt: maxt, + isoState: is, + }, nil +} + +type headChunkReader struct { + head *Head + mint, maxt int64 + isoState *isolationState +} + +func (h *headChunkReader) Close() error { + h.isoState.Close() + return nil +} + +// Chunk returns the chunk for the reference number. +func (h *headChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) { + sid, cid := chunks.HeadChunkRef(ref).Unpack() + + s := h.head.series.getByID(sid) + // This means that the series has been garbage collected. + if s == nil { + return nil, storage.ErrNotFound + } + + s.Lock() + c, garbageCollect, err := s.chunk(cid, h.head.chunkDiskMapper) + if err != nil { + s.Unlock() + return nil, err + } + defer func() { + if garbageCollect { + // Set this to nil so that Go GC can collect it after it has been used. + c.chunk = nil + s.memChunkPool.Put(c) + } + }() + + // This means that the chunk is outside the specified range. + if !c.OverlapsClosedInterval(h.mint, h.maxt) { + s.Unlock() + return nil, storage.ErrNotFound + } + s.Unlock() + + return &safeChunk{ + Chunk: c.chunk, + s: s, + cid: cid, + isoState: h.isoState, + chunkDiskMapper: h.head.chunkDiskMapper, + }, nil +} + +// chunk returns the chunk for the HeadChunkID from memory or by m-mapping it from the disk. +// If garbageCollect is true, it means that the returned *memChunk +// (and not the chunkenc.Chunk inside it) can be garbage collected after its usage. +func (s *memSeries) chunk(id chunks.HeadChunkID, chunkDiskMapper *chunks.ChunkDiskMapper) (chunk *memChunk, garbageCollect bool, err error) { + // ix represents the index of chunk in the s.mmappedChunks slice. The chunk id's are + // incremented by 1 when new chunk is created, hence (id - firstChunkID) gives the slice index. + // The max index for the s.mmappedChunks slice can be len(s.mmappedChunks)-1, hence if the ix + // is len(s.mmappedChunks), it represents the next chunk, which is the head chunk. + ix := int(id) - int(s.firstChunkID) + if ix < 0 || ix > len(s.mmappedChunks) { + return nil, false, storage.ErrNotFound + } + if ix == len(s.mmappedChunks) { + if s.headChunk == nil { + return nil, false, errors.New("invalid head chunk") + } + return s.headChunk, false, nil + } + chk, err := chunkDiskMapper.Chunk(s.mmappedChunks[ix].ref) + if err != nil { + if _, ok := err.(*chunks.CorruptionErr); ok { + panic(err) + } + return nil, false, err + } + mc := s.memChunkPool.Get().(*memChunk) + mc.chunk = chk + mc.minTime = s.mmappedChunks[ix].minTime + mc.maxTime = s.mmappedChunks[ix].maxTime + return mc, true, nil +} + +type safeChunk struct { + chunkenc.Chunk + s *memSeries + cid chunks.HeadChunkID + isoState *isolationState + chunkDiskMapper *chunks.ChunkDiskMapper +} + +func (c *safeChunk) Iterator(reuseIter chunkenc.Iterator) chunkenc.Iterator { + c.s.Lock() + it := c.s.iterator(c.cid, c.isoState, c.chunkDiskMapper, reuseIter) + c.s.Unlock() + return it +} + +// iterator returns a chunk iterator for the requested chunkID, or a NopIterator if the requested ID is out of range. +// It is unsafe to call this concurrently with s.append(...) without holding the series lock. +func (s *memSeries) iterator(id chunks.HeadChunkID, isoState *isolationState, chunkDiskMapper *chunks.ChunkDiskMapper, it chunkenc.Iterator) chunkenc.Iterator { + c, garbageCollect, err := s.chunk(id, chunkDiskMapper) + // TODO(fabxc): Work around! An error will be returns when a querier have retrieved a pointer to a + // series's chunk, which got then garbage collected before it got + // accessed. We must ensure to not garbage collect as long as any + // readers still hold a reference. + if err != nil { + return chunkenc.NewNopIterator() + } + defer func() { + if garbageCollect { + // Set this to nil so that Go GC can collect it after it has been used. + // This should be done always at the end. + c.chunk = nil + s.memChunkPool.Put(c) + } + }() + + ix := int(id) - int(s.firstChunkID) + + numSamples := c.chunk.NumSamples() + stopAfter := numSamples + + if isoState != nil && !isoState.IsolationDisabled() { + totalSamples := 0 // Total samples in this series. + previousSamples := 0 // Samples before this chunk. + + for j, d := range s.mmappedChunks { + totalSamples += int(d.numSamples) + if j < ix { + previousSamples += int(d.numSamples) + } + } + + if s.headChunk != nil { + totalSamples += s.headChunk.chunk.NumSamples() + } + + // Removing the extra transactionIDs that are relevant for samples that + // come after this chunk, from the total transactionIDs. + appendIDsToConsider := s.txs.txIDCount - (totalSamples - (previousSamples + numSamples)) + + // Iterate over the appendIDs, find the first one that the isolation state says not + // to return. + it := s.txs.iterator() + for index := 0; index < appendIDsToConsider; index++ { + appendID := it.At() + if appendID <= isoState.maxAppendID { // Easy check first. + if _, ok := isoState.incompleteAppends[appendID]; !ok { + it.Next() + continue + } + } + stopAfter = numSamples - (appendIDsToConsider - index) + if stopAfter < 0 { + stopAfter = 0 // Stopped in a previous chunk. + } + break + } + } + + if stopAfter == 0 { + return chunkenc.NewNopIterator() + } + + if int(id)-int(s.firstChunkID) < len(s.mmappedChunks) { + if stopAfter == numSamples { + return c.chunk.Iterator(it) + } + if msIter, ok := it.(*stopIterator); ok { + msIter.Iterator = c.chunk.Iterator(msIter.Iterator) + msIter.i = -1 + msIter.stopAfter = stopAfter + return msIter + } + return &stopIterator{ + Iterator: c.chunk.Iterator(it), + i: -1, + stopAfter: stopAfter, + } + } + // Serve the last 4 samples for the last chunk from the sample buffer + // as their compressed bytes may be mutated by added samples. + if msIter, ok := it.(*memSafeIterator); ok { + msIter.Iterator = c.chunk.Iterator(msIter.Iterator) + msIter.i = -1 + msIter.total = numSamples + msIter.stopAfter = stopAfter + msIter.buf = s.sampleBuf + return msIter + } + return &memSafeIterator{ + stopIterator: stopIterator{ + Iterator: c.chunk.Iterator(it), + i: -1, + stopAfter: stopAfter, + }, + total: numSamples, + buf: s.sampleBuf, + } +} + +// memSafeIterator returns values from the wrapped stopIterator +// except the last 4, which come from buf. +type memSafeIterator struct { + stopIterator + + total int + buf [4]sample +} + +func (it *memSafeIterator) Seek(t int64) bool { + if it.Err() != nil { + return false + } + + ts, _ := it.At() + + for t > ts || it.i == -1 { + if !it.Next() { + return false + } + ts, _ = it.At() + } + + return true +} + +func (it *memSafeIterator) Next() bool { + if it.i+1 >= it.stopAfter { + return false + } + it.i++ + if it.total-it.i > 4 { + return it.Iterator.Next() + } + return true +} + +func (it *memSafeIterator) At() (int64, float64) { + if it.total-it.i > 4 { + return it.Iterator.At() + } + s := it.buf[4-(it.total-it.i)] + return s.t, s.v +} + +// stopIterator wraps an Iterator, but only returns the first +// stopAfter values, if initialized with i=-1. +type stopIterator struct { + chunkenc.Iterator + + i, stopAfter int +} + +func (it *stopIterator) Next() bool { + if it.i+1 >= it.stopAfter { + return false + } + it.i++ + return it.Iterator.Next() +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go b/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go new file mode 100644 index 00000000000..d0fe685ffb1 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/head_wal.go @@ -0,0 +1,1050 @@ +// Copyright 2021 The Prometheus Authors +// 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 tsdb + +import ( + "fmt" + "io/ioutil" + "math" + "os" + "path/filepath" + "runtime" + "strconv" + "strings" + "sync" + "time" + + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "go.uber.org/atomic" + + "github.com/prometheus/prometheus/model/exemplar" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/encoding" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/prometheus/prometheus/tsdb/record" + "github.com/prometheus/prometheus/tsdb/tombstones" + "github.com/prometheus/prometheus/tsdb/wal" +) + +func (h *Head) loadWAL(r *wal.Reader, multiRef map[chunks.HeadSeriesRef]chunks.HeadSeriesRef, mmappedChunks map[chunks.HeadSeriesRef][]*mmappedChunk) (err error) { + // Track number of samples that referenced a series we don't know about + // for error reporting. + var unknownRefs atomic.Uint64 + var unknownExemplarRefs atomic.Uint64 + // Track number of series records that had overlapping m-map chunks. + var mmapOverlappingChunks uint64 + + // Start workers that each process samples for a partition of the series ID space. + var ( + wg sync.WaitGroup + n = runtime.GOMAXPROCS(0) + processors = make([]walSubsetProcessor, n) + exemplarsInput chan record.RefExemplar + + dec record.Decoder + shards = make([][]record.RefSample, n) + + decoded = make(chan interface{}, 10) + decodeErr, seriesCreationErr error + seriesPool = sync.Pool{ + New: func() interface{} { + return []record.RefSeries{} + }, + } + samplesPool = sync.Pool{ + New: func() interface{} { + return []record.RefSample{} + }, + } + tstonesPool = sync.Pool{ + New: func() interface{} { + return []tombstones.Stone{} + }, + } + exemplarsPool = sync.Pool{ + New: func() interface{} { + return []record.RefExemplar{} + }, + } + ) + + defer func() { + // For CorruptionErr ensure to terminate all workers before exiting. + _, ok := err.(*wal.CorruptionErr) + if ok || seriesCreationErr != nil { + for i := 0; i < n; i++ { + processors[i].closeAndDrain() + } + close(exemplarsInput) + wg.Wait() + } + }() + + wg.Add(n) + for i := 0; i < n; i++ { + processors[i].setup() + + go func(wp *walSubsetProcessor) { + unknown := wp.processWALSamples(h) + unknownRefs.Add(unknown) + wg.Done() + }(&processors[i]) + } + + wg.Add(1) + exemplarsInput = make(chan record.RefExemplar, 300) + go func(input <-chan record.RefExemplar) { + var err error + defer wg.Done() + for e := range input { + ms := h.series.getByID(e.Ref) + if ms == nil { + unknownExemplarRefs.Inc() + continue + } + + if e.T < h.minValidTime.Load() { + continue + } + // At the moment the only possible error here is out of order exemplars, which we shouldn't see when + // replaying the WAL, so lets just log the error if it's not that type. + err = h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{Ts: e.T, Value: e.V, Labels: e.Labels}) + if err != nil && err == storage.ErrOutOfOrderExemplar { + level.Warn(h.logger).Log("msg", "Unexpected error when replaying WAL on exemplar record", "err", err) + } + } + }(exemplarsInput) + + go func() { + defer close(decoded) + for r.Next() { + rec := r.Record() + switch dec.Type(rec) { + case record.Series: + series := seriesPool.Get().([]record.RefSeries)[:0] + series, err = dec.Series(rec, series) + if err != nil { + decodeErr = &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode series"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- series + case record.Samples: + samples := samplesPool.Get().([]record.RefSample)[:0] + samples, err = dec.Samples(rec, samples) + if err != nil { + decodeErr = &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode samples"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- samples + case record.Tombstones: + tstones := tstonesPool.Get().([]tombstones.Stone)[:0] + tstones, err = dec.Tombstones(rec, tstones) + if err != nil { + decodeErr = &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode tombstones"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- tstones + case record.Exemplars: + exemplars := exemplarsPool.Get().([]record.RefExemplar)[:0] + exemplars, err = dec.Exemplars(rec, exemplars) + if err != nil { + decodeErr = &wal.CorruptionErr{ + Err: errors.Wrap(err, "decode exemplars"), + Segment: r.Segment(), + Offset: r.Offset(), + } + return + } + decoded <- exemplars + default: + // Noop. + } + } + }() + + // The records are always replayed from the oldest to the newest. +Outer: + for d := range decoded { + switch v := d.(type) { + case []record.RefSeries: + for _, walSeries := range v { + mSeries, created, err := h.getOrCreateWithID(walSeries.Ref, walSeries.Labels.Hash(), walSeries.Labels) + if err != nil { + seriesCreationErr = err + break Outer + } + + if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < walSeries.Ref { + h.lastSeriesID.Store(uint64(walSeries.Ref)) + } + + idx := uint64(mSeries.ref) % uint64(n) + // It is possible that some old sample is being processed in processWALSamples that + // could cause race below. So we wait for the goroutine to empty input the buffer and finish + // processing all old samples after emptying the buffer. + processors[idx].waitUntilIdle() + // Lock the subset so we can modify the series object + processors[idx].mx.Lock() + + mmc := mmappedChunks[walSeries.Ref] + + if created { + // This is the first WAL series record for this series. + h.resetSeriesWithMMappedChunks(mSeries, mmc) + processors[idx].mx.Unlock() + continue + } + + // There's already a different ref for this series. + // A duplicate series record is only possible when the old samples were already compacted into a block. + // Hence we can discard all the samples and m-mapped chunks replayed till now for this series. + + multiRef[walSeries.Ref] = mSeries.ref + + // Checking if the new m-mapped chunks overlap with the already existing ones. + if len(mSeries.mmappedChunks) > 0 && len(mmc) > 0 { + if overlapsClosedInterval( + mSeries.mmappedChunks[0].minTime, + mSeries.mmappedChunks[len(mSeries.mmappedChunks)-1].maxTime, + mmc[0].minTime, + mmc[len(mmc)-1].maxTime, + ) { + mmapOverlappingChunks++ + level.Debug(h.logger).Log( + "msg", "M-mapped chunks overlap on a duplicate series record", + "series", mSeries.lset.String(), + "oldref", mSeries.ref, + "oldmint", mSeries.mmappedChunks[0].minTime, + "oldmaxt", mSeries.mmappedChunks[len(mSeries.mmappedChunks)-1].maxTime, + "newref", walSeries.Ref, + "newmint", mmc[0].minTime, + "newmaxt", mmc[len(mmc)-1].maxTime, + ) + } + } + + // Replacing m-mapped chunks with the new ones (could be empty). + h.resetSeriesWithMMappedChunks(mSeries, mmc) + + processors[idx].mx.Unlock() + } + //nolint:staticcheck // Ignore SA6002 relax staticcheck verification. + seriesPool.Put(v) + case []record.RefSample: + samples := v + // We split up the samples into chunks of 5000 samples or less. + // With O(300 * #cores) in-flight sample batches, large scrapes could otherwise + // cause thousands of very large in flight buffers occupying large amounts + // of unused memory. + for len(samples) > 0 { + m := 5000 + if len(samples) < m { + m = len(samples) + } + for i := 0; i < n; i++ { + shards[i] = processors[i].reuseBuf() + } + for _, sam := range samples[:m] { + if r, ok := multiRef[sam.Ref]; ok { + sam.Ref = r + } + mod := uint64(sam.Ref) % uint64(n) + shards[mod] = append(shards[mod], sam) + } + for i := 0; i < n; i++ { + processors[i].input <- shards[i] + } + samples = samples[m:] + } + //nolint:staticcheck // Ignore SA6002 relax staticcheck verification. + samplesPool.Put(v) + case []tombstones.Stone: + for _, s := range v { + for _, itv := range s.Intervals { + if itv.Maxt < h.minValidTime.Load() { + continue + } + if m := h.series.getByID(chunks.HeadSeriesRef(s.Ref)); m == nil { + unknownRefs.Inc() + continue + } + h.tombstones.AddInterval(storage.SeriesRef(s.Ref), itv) + } + } + //nolint:staticcheck // Ignore SA6002 relax staticcheck verification. + tstonesPool.Put(v) + case []record.RefExemplar: + for _, e := range v { + exemplarsInput <- e + } + //nolint:staticcheck // Ignore SA6002 relax staticcheck verification. + exemplarsPool.Put(v) + default: + panic(fmt.Errorf("unexpected decoded type: %T", d)) + } + } + + if decodeErr != nil { + return decodeErr + } + if seriesCreationErr != nil { + // Drain the channel to unblock the goroutine. + for range decoded { + } + return seriesCreationErr + } + + // Signal termination to each worker and wait for it to close its output channel. + for i := 0; i < n; i++ { + processors[i].closeAndDrain() + } + close(exemplarsInput) + wg.Wait() + + if r.Err() != nil { + return errors.Wrap(r.Err(), "read records") + } + + if unknownRefs.Load() > 0 || unknownExemplarRefs.Load() > 0 { + level.Warn(h.logger).Log("msg", "Unknown series references", "samples", unknownRefs.Load(), "exemplars", unknownExemplarRefs.Load()) + } + if mmapOverlappingChunks > 0 { + level.Info(h.logger).Log("msg", "Overlapping m-map chunks on duplicate series records", "count", mmapOverlappingChunks) + } + return nil +} + +// resetSeriesWithMMappedChunks is only used during the WAL replay. +func (h *Head) resetSeriesWithMMappedChunks(mSeries *memSeries, mmc []*mmappedChunk) { + h.metrics.chunksCreated.Add(float64(len(mmc))) + h.metrics.chunksRemoved.Add(float64(len(mSeries.mmappedChunks))) + h.metrics.chunks.Add(float64(len(mmc) - len(mSeries.mmappedChunks))) + mSeries.mmappedChunks = mmc + // Cache the last mmapped chunk time, so we can skip calling append() for samples it will reject. + if len(mmc) == 0 { + mSeries.mmMaxTime = math.MinInt64 + } else { + mSeries.mmMaxTime = mmc[len(mmc)-1].maxTime + h.updateMinMaxTime(mmc[0].minTime, mSeries.mmMaxTime) + } + + // Any samples replayed till now would already be compacted. Resetting the head chunk. + mSeries.nextAt = 0 + mSeries.headChunk = nil + mSeries.app = nil +} + +type walSubsetProcessor struct { + mx sync.Mutex // Take this lock while modifying series in the subset. + input chan []record.RefSample + output chan []record.RefSample +} + +func (wp *walSubsetProcessor) setup() { + wp.output = make(chan []record.RefSample, 300) + wp.input = make(chan []record.RefSample, 300) +} + +func (wp *walSubsetProcessor) closeAndDrain() { + close(wp.input) + for range wp.output { + } +} + +// If there is a buffer in the output chan, return it for reuse, otherwise return nil. +func (wp *walSubsetProcessor) reuseBuf() []record.RefSample { + select { + case buf := <-wp.output: + return buf[:0] + default: + } + return nil +} + +// processWALSamples adds the samples it receives to the head and passes +// the buffer received to an output channel for reuse. +// Samples before the minValidTime timestamp are discarded. +func (wp *walSubsetProcessor) processWALSamples(h *Head) (unknownRefs uint64) { + defer close(wp.output) + + minValidTime := h.minValidTime.Load() + mint, maxt := int64(math.MaxInt64), int64(math.MinInt64) + + for samples := range wp.input { + wp.mx.Lock() + for _, s := range samples { + if s.T < minValidTime { + continue + } + ms := h.series.getByID(s.Ref) + if ms == nil { + unknownRefs++ + continue + } + if s.T <= ms.mmMaxTime { + continue + } + if _, chunkCreated := ms.append(s.T, s.V, 0, h.chunkDiskMapper); chunkCreated { + h.metrics.chunksCreated.Inc() + h.metrics.chunks.Inc() + } + if s.T > maxt { + maxt = s.T + } + if s.T < mint { + mint = s.T + } + } + wp.mx.Unlock() + wp.output <- samples + } + h.updateMinMaxTime(mint, maxt) + + return unknownRefs +} + +func (wp *walSubsetProcessor) waitUntilIdle() { + select { + case <-wp.output: // Allow output side to drain to avoid deadlock. + default: + } + wp.input <- []record.RefSample{} + for len(wp.input) != 0 { + time.Sleep(1 * time.Millisecond) + select { + case <-wp.output: // Allow output side to drain to avoid deadlock. + default: + } + } +} + +const ( + chunkSnapshotRecordTypeSeries uint8 = 1 + chunkSnapshotRecordTypeTombstones uint8 = 2 + chunkSnapshotRecordTypeExemplars uint8 = 3 +) + +type chunkSnapshotRecord struct { + ref chunks.HeadSeriesRef + lset labels.Labels + chunkRange int64 + mc *memChunk + sampleBuf [4]sample +} + +func (s *memSeries) encodeToSnapshotRecord(b []byte) []byte { + buf := encoding.Encbuf{B: b} + + buf.PutByte(chunkSnapshotRecordTypeSeries) + buf.PutBE64(uint64(s.ref)) + buf.PutUvarint(len(s.lset)) + for _, l := range s.lset { + buf.PutUvarintStr(l.Name) + buf.PutUvarintStr(l.Value) + } + buf.PutBE64int64(s.chunkRange) + + s.Lock() + if s.headChunk == nil { + buf.PutUvarint(0) + } else { + buf.PutUvarint(1) + buf.PutBE64int64(s.headChunk.minTime) + buf.PutBE64int64(s.headChunk.maxTime) + buf.PutByte(byte(s.headChunk.chunk.Encoding())) + buf.PutUvarintBytes(s.headChunk.chunk.Bytes()) + // Put the sample buf. + for _, smpl := range s.sampleBuf { + buf.PutBE64int64(smpl.t) + buf.PutBEFloat64(smpl.v) + } + } + s.Unlock() + + return buf.Get() +} + +func decodeSeriesFromChunkSnapshot(b []byte) (csr chunkSnapshotRecord, err error) { + dec := encoding.Decbuf{B: b} + + if flag := dec.Byte(); flag != chunkSnapshotRecordTypeSeries { + return csr, errors.Errorf("invalid record type %x", flag) + } + + csr.ref = chunks.HeadSeriesRef(dec.Be64()) + + // The label set written to the disk is already sorted. + csr.lset = make(labels.Labels, dec.Uvarint()) + for i := range csr.lset { + csr.lset[i].Name = dec.UvarintStr() + csr.lset[i].Value = dec.UvarintStr() + } + + csr.chunkRange = dec.Be64int64() + if dec.Uvarint() == 0 { + return + } + + csr.mc = &memChunk{} + csr.mc.minTime = dec.Be64int64() + csr.mc.maxTime = dec.Be64int64() + enc := chunkenc.Encoding(dec.Byte()) + + // The underlying bytes gets re-used later, so make a copy. + chunkBytes := dec.UvarintBytes() + chunkBytesCopy := make([]byte, len(chunkBytes)) + copy(chunkBytesCopy, chunkBytes) + + chk, err := chunkenc.FromData(enc, chunkBytesCopy) + if err != nil { + return csr, errors.Wrap(err, "chunk from data") + } + csr.mc.chunk = chk + + for i := range csr.sampleBuf { + csr.sampleBuf[i].t = dec.Be64int64() + csr.sampleBuf[i].v = dec.Be64Float64() + } + + err = dec.Err() + if err != nil && len(dec.B) > 0 { + err = errors.Errorf("unexpected %d bytes left in entry", len(dec.B)) + } + + return +} + +func encodeTombstonesToSnapshotRecord(tr tombstones.Reader) ([]byte, error) { + buf := encoding.Encbuf{} + + buf.PutByte(chunkSnapshotRecordTypeTombstones) + b, err := tombstones.Encode(tr) + if err != nil { + return nil, errors.Wrap(err, "encode tombstones") + } + buf.PutUvarintBytes(b) + + return buf.Get(), nil +} + +func decodeTombstonesSnapshotRecord(b []byte) (tombstones.Reader, error) { + dec := encoding.Decbuf{B: b} + + if flag := dec.Byte(); flag != chunkSnapshotRecordTypeTombstones { + return nil, errors.Errorf("invalid record type %x", flag) + } + + tr, err := tombstones.Decode(dec.UvarintBytes()) + return tr, errors.Wrap(err, "decode tombstones") +} + +const chunkSnapshotPrefix = "chunk_snapshot." + +// ChunkSnapshot creates a snapshot of all the series and tombstones in the head. +// It deletes the old chunk snapshots if the chunk snapshot creation is successful. +// +// The chunk snapshot is stored in a directory named chunk_snapshot.N.M and is written +// using the WAL package. N is the last WAL segment present during snapshotting and +// M is the offset in segment N upto which data was written. +// +// The snapshot first contains all series (each in individual records and not sorted), followed by +// tombstones (a single record), and finally exemplars (>= 1 record). Exemplars are in the order they +// were written to the circular buffer. +func (h *Head) ChunkSnapshot() (*ChunkSnapshotStats, error) { + if h.wal == nil { + // If we are not storing any WAL, does not make sense to take a snapshot too. + level.Warn(h.logger).Log("msg", "skipping chunk snapshotting as WAL is disabled") + return &ChunkSnapshotStats{}, nil + } + h.chunkSnapshotMtx.Lock() + defer h.chunkSnapshotMtx.Unlock() + + stats := &ChunkSnapshotStats{} + + wlast, woffset, err := h.wal.LastSegmentAndOffset() + if err != nil && err != record.ErrNotFound { + return stats, errors.Wrap(err, "get last wal segment and offset") + } + + _, cslast, csoffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot) + if err != nil && err != record.ErrNotFound { + return stats, errors.Wrap(err, "find last chunk snapshot") + } + + if wlast == cslast && woffset == csoffset { + // Nothing has been written to the WAL/Head since the last snapshot. + return stats, nil + } + + snapshotName := chunkSnapshotDir(wlast, woffset) + + cpdir := filepath.Join(h.opts.ChunkDirRoot, snapshotName) + cpdirtmp := cpdir + ".tmp" + stats.Dir = cpdir + + if err := os.MkdirAll(cpdirtmp, 0o777); err != nil { + return stats, errors.Wrap(err, "create chunk snapshot dir") + } + cp, err := wal.New(nil, nil, cpdirtmp, h.wal.CompressionEnabled()) + if err != nil { + return stats, errors.Wrap(err, "open chunk snapshot") + } + + // Ensures that an early return caused by an error doesn't leave any tmp files. + defer func() { + cp.Close() + os.RemoveAll(cpdirtmp) + }() + + var ( + buf []byte + recs [][]byte + ) + // Add all series to the snapshot. + stripeSize := h.series.size + for i := 0; i < stripeSize; i++ { + h.series.locks[i].RLock() + + for _, s := range h.series.series[i] { + start := len(buf) + buf = s.encodeToSnapshotRecord(buf) + if len(buf[start:]) == 0 { + continue // All contents discarded. + } + recs = append(recs, buf[start:]) + // Flush records in 10 MB increments. + if len(buf) > 10*1024*1024 { + if err := cp.Log(recs...); err != nil { + h.series.locks[i].RUnlock() + return stats, errors.Wrap(err, "flush records") + } + buf, recs = buf[:0], recs[:0] + } + } + stats.TotalSeries += len(h.series.series[i]) + + h.series.locks[i].RUnlock() + } + + // Add tombstones to the snapshot. + tombstonesReader, err := h.Tombstones() + if err != nil { + return stats, errors.Wrap(err, "get tombstones") + } + rec, err := encodeTombstonesToSnapshotRecord(tombstonesReader) + if err != nil { + return stats, errors.Wrap(err, "encode tombstones") + } + recs = append(recs, rec) + // Flush remaining series records and tombstones. + if err := cp.Log(recs...); err != nil { + return stats, errors.Wrap(err, "flush records") + } + buf = buf[:0] + + // Add exemplars in the snapshot. + // We log in batches, with each record having upto 10000 exemplars. + // Assuming 100 bytes (overestimate) per exemplar, that's ~1MB. + maxExemplarsPerRecord := 10000 + batch := make([]record.RefExemplar, 0, maxExemplarsPerRecord) + enc := record.Encoder{} + flushExemplars := func() error { + if len(batch) == 0 { + return nil + } + buf = buf[:0] + encbuf := encoding.Encbuf{B: buf} + encbuf.PutByte(chunkSnapshotRecordTypeExemplars) + enc.EncodeExemplarsIntoBuffer(batch, &encbuf) + if err := cp.Log(encbuf.Get()); err != nil { + return errors.Wrap(err, "log exemplars") + } + buf, batch = buf[:0], batch[:0] + return nil + } + err = h.exemplars.IterateExemplars(func(seriesLabels labels.Labels, e exemplar.Exemplar) error { + if len(batch) >= maxExemplarsPerRecord { + if err := flushExemplars(); err != nil { + return errors.Wrap(err, "flush exemplars") + } + } + + ms := h.series.getByHash(seriesLabels.Hash(), seriesLabels) + if ms == nil { + // It is possible that exemplar refers to some old series. We discard such exemplars. + return nil + } + batch = append(batch, record.RefExemplar{ + Ref: ms.ref, + T: e.Ts, + V: e.Value, + Labels: e.Labels, + }) + return nil + }) + if err != nil { + return stats, errors.Wrap(err, "iterate exemplars") + } + + // Flush remaining exemplars. + if err := flushExemplars(); err != nil { + return stats, errors.Wrap(err, "flush exemplars at the end") + } + + if err := cp.Close(); err != nil { + return stats, errors.Wrap(err, "close chunk snapshot") + } + if err := fileutil.Replace(cpdirtmp, cpdir); err != nil { + return stats, errors.Wrap(err, "rename chunk snapshot directory") + } + + if err := DeleteChunkSnapshots(h.opts.ChunkDirRoot, wlast, woffset); err != nil { + // Leftover old chunk snapshots do not cause problems down the line beyond + // occupying disk space. + // They will just be ignored since a higher chunk snapshot exists. + level.Error(h.logger).Log("msg", "delete old chunk snapshots", "err", err) + } + return stats, nil +} + +func chunkSnapshotDir(wlast, woffset int) string { + return fmt.Sprintf(chunkSnapshotPrefix+"%06d.%010d", wlast, woffset) +} + +func (h *Head) performChunkSnapshot() error { + level.Info(h.logger).Log("msg", "creating chunk snapshot") + startTime := time.Now() + stats, err := h.ChunkSnapshot() + elapsed := time.Since(startTime) + if err == nil { + level.Info(h.logger).Log("msg", "chunk snapshot complete", "duration", elapsed.String(), "num_series", stats.TotalSeries, "dir", stats.Dir) + } + return errors.Wrap(err, "chunk snapshot") +} + +// ChunkSnapshotStats returns stats about a created chunk snapshot. +type ChunkSnapshotStats struct { + TotalSeries int + Dir string +} + +// LastChunkSnapshot returns the directory name and index of the most recent chunk snapshot. +// If dir does not contain any chunk snapshots, ErrNotFound is returned. +func LastChunkSnapshot(dir string) (string, int, int, error) { + files, err := ioutil.ReadDir(dir) + if err != nil { + return "", 0, 0, err + } + maxIdx, maxOffset := -1, -1 + maxFileName := "" + for i := 0; i < len(files); i++ { + fi := files[i] + + if !strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) { + continue + } + if !fi.IsDir() { + return "", 0, 0, errors.Errorf("chunk snapshot %s is not a directory", fi.Name()) + } + + splits := strings.Split(fi.Name()[len(chunkSnapshotPrefix):], ".") + if len(splits) != 2 { + // Chunk snapshots is not in the right format, we do not care about it. + continue + } + + idx, err := strconv.Atoi(splits[0]) + if err != nil { + continue + } + + offset, err := strconv.Atoi(splits[1]) + if err != nil { + continue + } + + if idx > maxIdx || (idx == maxIdx && offset > maxOffset) { + maxIdx, maxOffset = idx, offset + maxFileName = filepath.Join(dir, fi.Name()) + } + } + if maxFileName == "" { + return "", 0, 0, record.ErrNotFound + } + return maxFileName, maxIdx, maxOffset, nil +} + +// DeleteChunkSnapshots deletes all chunk snapshots in a directory below a given index. +func DeleteChunkSnapshots(dir string, maxIndex, maxOffset int) error { + files, err := ioutil.ReadDir(dir) + if err != nil { + return err + } + + errs := tsdb_errors.NewMulti() + for _, fi := range files { + if !strings.HasPrefix(fi.Name(), chunkSnapshotPrefix) { + continue + } + + splits := strings.Split(fi.Name()[len(chunkSnapshotPrefix):], ".") + if len(splits) != 2 { + continue + } + + idx, err := strconv.Atoi(splits[0]) + if err != nil { + continue + } + + offset, err := strconv.Atoi(splits[1]) + if err != nil { + continue + } + + if idx < maxIndex || (idx == maxIndex && offset < maxOffset) { + if err := os.RemoveAll(filepath.Join(dir, fi.Name())); err != nil { + errs.Add(err) + } + } + + } + return errs.Err() +} + +// loadChunkSnapshot replays the chunk snapshot and restores the Head state from it. If there was any error returned, +// it is the responsibility of the caller to clear the contents of the Head. +func (h *Head) loadChunkSnapshot() (int, int, map[chunks.HeadSeriesRef]*memSeries, error) { + dir, snapIdx, snapOffset, err := LastChunkSnapshot(h.opts.ChunkDirRoot) + if err != nil { + if err == record.ErrNotFound { + return snapIdx, snapOffset, nil, nil + } + return snapIdx, snapOffset, nil, errors.Wrap(err, "find last chunk snapshot") + } + + start := time.Now() + sr, err := wal.NewSegmentsReader(dir) + if err != nil { + return snapIdx, snapOffset, nil, errors.Wrap(err, "open chunk snapshot") + } + defer func() { + if err := sr.Close(); err != nil { + level.Warn(h.logger).Log("msg", "error while closing the wal segments reader", "err", err) + } + }() + + var ( + numSeries = 0 + unknownRefs = int64(0) + n = runtime.GOMAXPROCS(0) + wg sync.WaitGroup + recordChan = make(chan chunkSnapshotRecord, 5*n) + shardedRefSeries = make([]map[chunks.HeadSeriesRef]*memSeries, n) + errChan = make(chan error, n) + refSeries map[chunks.HeadSeriesRef]*memSeries + exemplarBuf []record.RefExemplar + dec record.Decoder + ) + + wg.Add(n) + for i := 0; i < n; i++ { + go func(idx int, rc <-chan chunkSnapshotRecord) { + defer wg.Done() + defer func() { + // If there was an error, drain the channel + // to unblock the main thread. + for range rc { + } + }() + + shardedRefSeries[idx] = make(map[chunks.HeadSeriesRef]*memSeries) + localRefSeries := shardedRefSeries[idx] + + for csr := range rc { + series, _, err := h.getOrCreateWithID(csr.ref, csr.lset.Hash(), csr.lset) + if err != nil { + errChan <- err + return + } + localRefSeries[csr.ref] = series + if chunks.HeadSeriesRef(h.lastSeriesID.Load()) < series.ref { + h.lastSeriesID.Store(uint64(series.ref)) + } + + series.chunkRange = csr.chunkRange + if csr.mc == nil { + continue + } + series.nextAt = csr.mc.maxTime // This will create a new chunk on append. + series.headChunk = csr.mc + for i := range series.sampleBuf { + series.sampleBuf[i].t = csr.sampleBuf[i].t + series.sampleBuf[i].v = csr.sampleBuf[i].v + } + + app, err := series.headChunk.chunk.Appender() + if err != nil { + errChan <- err + return + } + series.app = app + + h.updateMinMaxTime(csr.mc.minTime, csr.mc.maxTime) + } + }(i, recordChan) + } + + r := wal.NewReader(sr) + var loopErr error +Outer: + for r.Next() { + select { + case err := <-errChan: + errChan <- err + break Outer + default: + } + + rec := r.Record() + switch rec[0] { + case chunkSnapshotRecordTypeSeries: + numSeries++ + csr, err := decodeSeriesFromChunkSnapshot(rec) + if err != nil { + loopErr = errors.Wrap(err, "decode series record") + break Outer + } + recordChan <- csr + + case chunkSnapshotRecordTypeTombstones: + tr, err := decodeTombstonesSnapshotRecord(rec) + if err != nil { + loopErr = errors.Wrap(err, "decode tombstones") + break Outer + } + + if err = tr.Iter(func(ref storage.SeriesRef, ivs tombstones.Intervals) error { + h.tombstones.AddInterval(ref, ivs...) + return nil + }); err != nil { + loopErr = errors.Wrap(err, "iterate tombstones") + break Outer + } + + case chunkSnapshotRecordTypeExemplars: + // Exemplars are at the end of snapshot. So all series are loaded at this point. + if len(refSeries) == 0 { + close(recordChan) + wg.Wait() + + refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries) + for _, shard := range shardedRefSeries { + for k, v := range shard { + refSeries[k] = v + } + } + } + + if !h.opts.EnableExemplarStorage || h.opts.MaxExemplars.Load() <= 0 { + // Exemplar storage is disabled. + continue Outer + } + + decbuf := encoding.Decbuf{B: rec[1:]} + + exemplarBuf = exemplarBuf[:0] + exemplarBuf, err = dec.ExemplarsFromBuffer(&decbuf, exemplarBuf) + if err != nil { + loopErr = errors.Wrap(err, "exemplars from buffer") + break Outer + } + + for _, e := range exemplarBuf { + ms, ok := refSeries[e.Ref] + if !ok { + unknownRefs++ + continue + } + + if err := h.exemplars.AddExemplar(ms.lset, exemplar.Exemplar{ + Labels: e.Labels, + Value: e.V, + Ts: e.T, + }); err != nil { + loopErr = errors.Wrap(err, "add exemplar") + break Outer + } + } + + default: + // This is a record type we don't understand. It is either and old format from earlier versions, + // or a new format and the code was rolled back to old version. + loopErr = errors.Errorf("unsuported snapshot record type 0b%b", rec[0]) + break Outer + } + } + if len(refSeries) == 0 { + close(recordChan) + wg.Wait() + } + + close(errChan) + merr := tsdb_errors.NewMulti(errors.Wrap(loopErr, "decode loop")) + for err := range errChan { + merr.Add(errors.Wrap(err, "record processing")) + } + if err := merr.Err(); err != nil { + return -1, -1, nil, err + } + + if r.Err() != nil { + return -1, -1, nil, errors.Wrap(r.Err(), "read records") + } + + if len(refSeries) == 0 { + // We had no exemplar record, so we have to build the map here. + refSeries = make(map[chunks.HeadSeriesRef]*memSeries, numSeries) + for _, shard := range shardedRefSeries { + for k, v := range shard { + refSeries[k] = v + } + } + } + + elapsed := time.Since(start) + level.Info(h.logger).Log("msg", "chunk snapshot loaded", "dir", dir, "num_series", numSeries, "duration", elapsed.String()) + if unknownRefs > 0 { + level.Warn(h.logger).Log("msg", "unknown series references during chunk snapshot replay", "count", unknownRefs) + } + + return snapIdx, snapOffset, refSeries, nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/isolation.go b/vendor/github.com/prometheus/prometheus/tsdb/isolation.go new file mode 100644 index 00000000000..4919bfe9156 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/isolation.go @@ -0,0 +1,292 @@ +// Copyright 2020 The Prometheus Authors +// 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 tsdb + +import ( + "sync" +) + +// isolationState holds the isolation information. +type isolationState struct { + // We will ignore all appends above the max, or that are incomplete. + maxAppendID uint64 + incompleteAppends map[uint64]struct{} + lowWatermark uint64 // Lowest of incompleteAppends/maxAppendID. + isolation *isolation + mint, maxt int64 // Time ranges of the read. + + // Doubly linked list of active reads. + next *isolationState + prev *isolationState +} + +// Close closes the state. +func (i *isolationState) Close() { + i.isolation.readMtx.Lock() + defer i.isolation.readMtx.Unlock() + i.next.prev = i.prev + i.prev.next = i.next +} + +func (i *isolationState) IsolationDisabled() bool { + return i.isolation.disabled +} + +type isolationAppender struct { + appendID uint64 + prev *isolationAppender + next *isolationAppender +} + +// isolation is the global isolation state. +type isolation struct { + // Mutex for accessing lastAppendID and appendsOpen. + appendMtx sync.RWMutex + // Which appends are currently in progress. + appendsOpen map[uint64]*isolationAppender + // New appenders with higher appendID are added to the end. First element keeps lastAppendId. + // appendsOpenList.next points to the first element and appendsOpenList.prev points to the last element. + // If there are no appenders, both point back to appendsOpenList. + appendsOpenList *isolationAppender + // Pool of reusable *isolationAppender to save on allocations. + appendersPool sync.Pool + + // Mutex for accessing readsOpen. + // If taking both appendMtx and readMtx, take appendMtx first. + readMtx sync.RWMutex + // All current in use isolationStates. This is a doubly-linked list. + readsOpen *isolationState + // If true, writes are not tracked while reads are still tracked. + disabled bool +} + +func newIsolation(disabled bool) *isolation { + isoState := &isolationState{} + isoState.next = isoState + isoState.prev = isoState + + appender := &isolationAppender{} + appender.next = appender + appender.prev = appender + + return &isolation{ + appendsOpen: map[uint64]*isolationAppender{}, + appendsOpenList: appender, + readsOpen: isoState, + disabled: disabled, + appendersPool: sync.Pool{New: func() interface{} { return &isolationAppender{} }}, + } +} + +// lowWatermark returns the appendID below which we no longer need to track +// which appends were from which appendID. +func (i *isolation) lowWatermark() uint64 { + if i.disabled { + return 0 + } + + i.appendMtx.RLock() // Take appendMtx first. + defer i.appendMtx.RUnlock() + return i.lowWatermarkLocked() +} + +func (i *isolation) lowWatermarkLocked() uint64 { + if i.disabled { + return 0 + } + + i.readMtx.RLock() + defer i.readMtx.RUnlock() + if i.readsOpen.prev != i.readsOpen { + return i.readsOpen.prev.lowWatermark + } + + // Lowest appendID from appenders, or lastAppendId. + return i.appendsOpenList.next.appendID +} + +// State returns an object used to control isolation +// between a query and appends. Must be closed when complete. +func (i *isolation) State(mint, maxt int64) *isolationState { + i.appendMtx.RLock() // Take append mutex before read mutex. + defer i.appendMtx.RUnlock() + + // We need to track the reads even when isolation is disabled. + isoState := &isolationState{ + maxAppendID: i.appendsOpenList.appendID, + lowWatermark: i.appendsOpenList.next.appendID, // Lowest appendID from appenders, or lastAppendId. + incompleteAppends: make(map[uint64]struct{}, len(i.appendsOpen)), + isolation: i, + mint: mint, + maxt: maxt, + } + for k := range i.appendsOpen { + isoState.incompleteAppends[k] = struct{}{} + } + + i.readMtx.Lock() + defer i.readMtx.Unlock() + isoState.prev = i.readsOpen + isoState.next = i.readsOpen.next + i.readsOpen.next.prev = isoState + i.readsOpen.next = isoState + + return isoState +} + +// TraverseOpenReads iterates through the open reads and runs the given +// function on those states. The given function MUST NOT mutate the isolationState. +// The iteration is stopped when the function returns false or once all reads have been iterated. +func (i *isolation) TraverseOpenReads(f func(s *isolationState) bool) { + i.readMtx.RLock() + defer i.readMtx.RUnlock() + s := i.readsOpen.next + for s != i.readsOpen { + if !f(s) { + return + } + s = s.next + } +} + +// newAppendID increments the transaction counter and returns a new transaction +// ID. The first ID returned is 1. +// Also returns the low watermark, to keep lock/unlock operations down. +func (i *isolation) newAppendID() (uint64, uint64) { + if i.disabled { + return 0, 0 + } + + i.appendMtx.Lock() + defer i.appendMtx.Unlock() + + // Last used appendID is stored in head element. + i.appendsOpenList.appendID++ + + app := i.appendersPool.Get().(*isolationAppender) + app.appendID = i.appendsOpenList.appendID + app.prev = i.appendsOpenList.prev + app.next = i.appendsOpenList + + i.appendsOpenList.prev.next = app + i.appendsOpenList.prev = app + + i.appendsOpen[app.appendID] = app + return app.appendID, i.lowWatermarkLocked() +} + +func (i *isolation) lastAppendID() uint64 { + if i.disabled { + return 0 + } + + i.appendMtx.RLock() + defer i.appendMtx.RUnlock() + + return i.appendsOpenList.appendID +} + +func (i *isolation) closeAppend(appendID uint64) { + if i.disabled { + return + } + + i.appendMtx.Lock() + defer i.appendMtx.Unlock() + + app := i.appendsOpen[appendID] + if app != nil { + app.prev.next = app.next + app.next.prev = app.prev + + delete(i.appendsOpen, appendID) + + // Clear all fields, and return to the pool. + *app = isolationAppender{} + i.appendersPool.Put(app) + } +} + +// The transactionID ring buffer. +type txRing struct { + txIDs []uint64 + txIDFirst int // Position of the first id in the ring. + txIDCount int // How many ids in the ring. +} + +func newTxRing(cap int) *txRing { + return &txRing{ + txIDs: make([]uint64, cap), + } +} + +func (txr *txRing) add(appendID uint64) { + if txr.txIDCount == len(txr.txIDs) { + // Ring buffer is full, expand by doubling. + newRing := make([]uint64, txr.txIDCount*2) + idx := copy(newRing[:], txr.txIDs[txr.txIDFirst:]) + copy(newRing[idx:], txr.txIDs[:txr.txIDFirst]) + txr.txIDs = newRing + txr.txIDFirst = 0 + } + + txr.txIDs[(txr.txIDFirst+txr.txIDCount)%len(txr.txIDs)] = appendID + txr.txIDCount++ +} + +func (txr *txRing) cleanupAppendIDsBelow(bound uint64) { + pos := txr.txIDFirst + + for txr.txIDCount > 0 { + if txr.txIDs[pos] < bound { + txr.txIDFirst++ + txr.txIDCount-- + } else { + break + } + + pos++ + if pos == len(txr.txIDs) { + pos = 0 + } + } + + txr.txIDFirst %= len(txr.txIDs) +} + +func (txr *txRing) iterator() *txRingIterator { + return &txRingIterator{ + pos: txr.txIDFirst, + ids: txr.txIDs, + } +} + +// txRingIterator lets you iterate over the ring. It doesn't terminate, +// it DOESN'T terminate. +type txRingIterator struct { + ids []uint64 + + pos int +} + +func (it *txRingIterator) At() uint64 { + return it.ids[it.pos] +} + +func (it *txRingIterator) Next() { + it.pos++ + if it.pos == len(it.ids) { + it.pos = 0 + } +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/querier.go b/vendor/github.com/prometheus/prometheus/tsdb/querier.go new file mode 100644 index 00000000000..522adb87cdd --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/querier.go @@ -0,0 +1,905 @@ +// Copyright 2017 The Prometheus Authors +// 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 tsdb + +import ( + "math" + "sort" + "strings" + "unicode/utf8" + + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/prometheus/prometheus/tsdb/tombstones" +) + +// Bitmap used by func isRegexMetaCharacter to check whether a character needs to be escaped. +var regexMetaCharacterBytes [16]byte + +// isRegexMetaCharacter reports whether byte b needs to be escaped. +func isRegexMetaCharacter(b byte) bool { + return b < utf8.RuneSelf && regexMetaCharacterBytes[b%16]&(1<<(b/16)) != 0 +} + +func init() { + for _, b := range []byte(`.+*?()|[]{}^$`) { + regexMetaCharacterBytes[b%16] |= 1 << (b / 16) + } +} + +type blockBaseQuerier struct { + index IndexReader + chunks ChunkReader + tombstones tombstones.Reader + + closed bool + + mint, maxt int64 +} + +func newBlockBaseQuerier(b BlockReader, mint, maxt int64) (*blockBaseQuerier, error) { + indexr, err := b.Index() + if err != nil { + return nil, errors.Wrap(err, "open index reader") + } + chunkr, err := b.Chunks() + if err != nil { + indexr.Close() + return nil, errors.Wrap(err, "open chunk reader") + } + tombsr, err := b.Tombstones() + if err != nil { + indexr.Close() + chunkr.Close() + return nil, errors.Wrap(err, "open tombstone reader") + } + + if tombsr == nil { + tombsr = tombstones.NewMemTombstones() + } + return &blockBaseQuerier{ + mint: mint, + maxt: maxt, + index: indexr, + chunks: chunkr, + tombstones: tombsr, + }, nil +} + +func (q *blockBaseQuerier) LabelValues(name string, matchers ...*labels.Matcher) ([]string, storage.Warnings, error) { + res, err := q.index.SortedLabelValues(name, matchers...) + return res, nil, err +} + +func (q *blockBaseQuerier) LabelNames(matchers ...*labels.Matcher) ([]string, storage.Warnings, error) { + res, err := q.index.LabelNames(matchers...) + return res, nil, err +} + +func (q *blockBaseQuerier) Close() error { + if q.closed { + return errors.New("block querier already closed") + } + + errs := tsdb_errors.NewMulti( + q.index.Close(), + q.chunks.Close(), + q.tombstones.Close(), + ) + q.closed = true + return errs.Err() +} + +type blockQuerier struct { + *blockBaseQuerier +} + +// NewBlockQuerier returns a querier against the block reader and requested min and max time range. +func NewBlockQuerier(b BlockReader, mint, maxt int64) (storage.Querier, error) { + q, err := newBlockBaseQuerier(b, mint, maxt) + if err != nil { + return nil, err + } + return &blockQuerier{blockBaseQuerier: q}, nil +} + +func (q *blockQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.SeriesSet { + mint := q.mint + maxt := q.maxt + disableTrimming := false + + p, err := PostingsForMatchers(q.index, ms...) + if err != nil { + return storage.ErrSeriesSet(err) + } + if sortSeries { + p = q.index.SortedPostings(p) + } + + if hints != nil { + mint = hints.Start + maxt = hints.End + disableTrimming = hints.DisableTrimming + if hints.Func == "series" { + // When you're only looking up metadata (for example series API), you don't need to load any chunks. + return newBlockSeriesSet(q.index, newNopChunkReader(), q.tombstones, p, mint, maxt, disableTrimming) + } + } + + return newBlockSeriesSet(q.index, q.chunks, q.tombstones, p, mint, maxt, disableTrimming) +} + +// blockChunkQuerier provides chunk querying access to a single block database. +type blockChunkQuerier struct { + *blockBaseQuerier +} + +// NewBlockChunkQuerier returns a chunk querier against the block reader and requested min and max time range. +func NewBlockChunkQuerier(b BlockReader, mint, maxt int64) (storage.ChunkQuerier, error) { + q, err := newBlockBaseQuerier(b, mint, maxt) + if err != nil { + return nil, err + } + return &blockChunkQuerier{blockBaseQuerier: q}, nil +} + +func (q *blockChunkQuerier) Select(sortSeries bool, hints *storage.SelectHints, ms ...*labels.Matcher) storage.ChunkSeriesSet { + mint := q.mint + maxt := q.maxt + disableTrimming := false + if hints != nil { + mint = hints.Start + maxt = hints.End + disableTrimming = hints.DisableTrimming + } + p, err := PostingsForMatchers(q.index, ms...) + if err != nil { + return storage.ErrChunkSeriesSet(err) + } + if sortSeries { + p = q.index.SortedPostings(p) + } + return newBlockChunkSeriesSet(q.index, q.chunks, q.tombstones, p, mint, maxt, disableTrimming) +} + +func findSetMatches(pattern string) []string { + // Return empty matches if the wrapper from Prometheus is missing. + if len(pattern) < 6 || pattern[:4] != "^(?:" || pattern[len(pattern)-2:] != ")$" { + return nil + } + escaped := false + sets := []*strings.Builder{{}} + for i := 4; i < len(pattern)-2; i++ { + if escaped { + switch { + case isRegexMetaCharacter(pattern[i]): + sets[len(sets)-1].WriteByte(pattern[i]) + case pattern[i] == '\\': + sets[len(sets)-1].WriteByte('\\') + default: + return nil + } + escaped = false + } else { + switch { + case isRegexMetaCharacter(pattern[i]): + if pattern[i] == '|' { + sets = append(sets, &strings.Builder{}) + } else { + return nil + } + case pattern[i] == '\\': + escaped = true + default: + sets[len(sets)-1].WriteByte(pattern[i]) + } + } + } + matches := make([]string, 0, len(sets)) + for _, s := range sets { + if s.Len() > 0 { + matches = append(matches, s.String()) + } + } + return matches +} + +// PostingsForMatchers assembles a single postings iterator against the index reader +// based on the given matchers. The resulting postings are not ordered by series. +func PostingsForMatchers(ix IndexReader, ms ...*labels.Matcher) (index.Postings, error) { + var its, notIts []index.Postings + // See which label must be non-empty. + // Optimization for case like {l=~".", l!="1"}. + labelMustBeSet := make(map[string]bool, len(ms)) + for _, m := range ms { + if !m.Matches("") { + labelMustBeSet[m.Name] = true + } + } + + for _, m := range ms { + if labelMustBeSet[m.Name] { + // If this matcher must be non-empty, we can be smarter. + matchesEmpty := m.Matches("") + isNot := m.Type == labels.MatchNotEqual || m.Type == labels.MatchNotRegexp + if isNot && matchesEmpty { // l!="foo" + // If the label can't be empty and is a Not and the inner matcher + // doesn't match empty, then subtract it out at the end. + inverse, err := m.Inverse() + if err != nil { + return nil, err + } + + it, err := postingsForMatcher(ix, inverse) + if err != nil { + return nil, err + } + notIts = append(notIts, it) + } else if isNot && !matchesEmpty { // l!="" + // If the label can't be empty and is a Not, but the inner matcher can + // be empty we need to use inversePostingsForMatcher. + inverse, err := m.Inverse() + if err != nil { + return nil, err + } + + it, err := inversePostingsForMatcher(ix, inverse) + if err != nil { + return nil, err + } + its = append(its, it) + } else { // l="a" + // Non-Not matcher, use normal postingsForMatcher. + it, err := postingsForMatcher(ix, m) + if err != nil { + return nil, err + } + its = append(its, it) + } + } else { // l="" + // If the matchers for a labelname selects an empty value, it selects all + // the series which don't have the label name set too. See: + // https://github.com/prometheus/prometheus/issues/3575 and + // https://github.com/prometheus/prometheus/pull/3578#issuecomment-351653555 + it, err := inversePostingsForMatcher(ix, m) + if err != nil { + return nil, err + } + notIts = append(notIts, it) + } + } + + // If there's nothing to subtract from, add in everything and remove the notIts later. + if len(its) == 0 && len(notIts) != 0 { + k, v := index.AllPostingsKey() + allPostings, err := ix.Postings(k, v) + if err != nil { + return nil, err + } + its = append(its, allPostings) + } + + it := index.Intersect(its...) + + for _, n := range notIts { + it = index.Without(it, n) + } + + return it, nil +} + +func postingsForMatcher(ix IndexReader, m *labels.Matcher) (index.Postings, error) { + // This method will not return postings for missing labels. + + // Fast-path for equal matching. + if m.Type == labels.MatchEqual { + return ix.Postings(m.Name, m.Value) + } + + // Fast-path for set matching. + if m.Type == labels.MatchRegexp { + setMatches := findSetMatches(m.GetRegexString()) + if len(setMatches) > 0 { + sort.Strings(setMatches) + return ix.Postings(m.Name, setMatches...) + } + } + + vals, err := ix.LabelValues(m.Name) + if err != nil { + return nil, err + } + + var res []string + lastVal, isSorted := "", true + for _, val := range vals { + if m.Matches(val) { + res = append(res, val) + if isSorted && val < lastVal { + isSorted = false + } + lastVal = val + } + } + + if len(res) == 0 { + return index.EmptyPostings(), nil + } + + if !isSorted { + sort.Strings(res) + } + return ix.Postings(m.Name, res...) +} + +// inversePostingsForMatcher returns the postings for the series with the label name set but not matching the matcher. +func inversePostingsForMatcher(ix IndexReader, m *labels.Matcher) (index.Postings, error) { + vals, err := ix.LabelValues(m.Name) + if err != nil { + return nil, err + } + + var res []string + lastVal, isSorted := "", true + for _, val := range vals { + if !m.Matches(val) { + res = append(res, val) + if isSorted && val < lastVal { + isSorted = false + } + lastVal = val + } + } + + if !isSorted { + sort.Strings(res) + } + return ix.Postings(m.Name, res...) +} + +func labelValuesWithMatchers(r IndexReader, name string, matchers ...*labels.Matcher) ([]string, error) { + p, err := PostingsForMatchers(r, matchers...) + if err != nil { + return nil, errors.Wrap(err, "fetching postings for matchers") + } + + allValues, err := r.LabelValues(name) + if err != nil { + return nil, errors.Wrapf(err, "fetching values of label %s", name) + } + valuesPostings := make([]index.Postings, len(allValues)) + for i, value := range allValues { + valuesPostings[i], err = r.Postings(name, value) + if err != nil { + return nil, errors.Wrapf(err, "fetching postings for %s=%q", name, value) + } + } + indexes, err := index.FindIntersectingPostings(p, valuesPostings) + if err != nil { + return nil, errors.Wrap(err, "intersecting postings") + } + + values := make([]string, 0, len(indexes)) + for _, idx := range indexes { + values = append(values, allValues[idx]) + } + + return values, nil +} + +func labelNamesWithMatchers(r IndexReader, matchers ...*labels.Matcher) ([]string, error) { + p, err := PostingsForMatchers(r, matchers...) + if err != nil { + return nil, err + } + + var postings []storage.SeriesRef + for p.Next() { + postings = append(postings, p.At()) + } + if p.Err() != nil { + return nil, errors.Wrapf(p.Err(), "postings for label names with matchers") + } + + return r.LabelNamesFor(postings...) +} + +// blockBaseSeriesSet allows to iterate over all series in the single block. +// Iterated series are trimmed with given min and max time as well as tombstones. +// See newBlockSeriesSet and newBlockChunkSeriesSet to use it for either sample or chunk iterating. +type blockBaseSeriesSet struct { + p index.Postings + index IndexReader + chunks ChunkReader + tombstones tombstones.Reader + mint, maxt int64 + disableTrimming bool + + currIterFn func() *populateWithDelGenericSeriesIterator + currLabels labels.Labels + + bufChks []chunks.Meta + bufLbls labels.Labels + err error +} + +func (b *blockBaseSeriesSet) Next() bool { + for b.p.Next() { + if err := b.index.Series(b.p.At(), &b.bufLbls, &b.bufChks); err != nil { + // Postings may be stale. Skip if no underlying series exists. + if errors.Cause(err) == storage.ErrNotFound { + continue + } + b.err = errors.Wrapf(err, "get series %d", b.p.At()) + return false + } + + if len(b.bufChks) == 0 { + continue + } + + intervals, err := b.tombstones.Get(b.p.At()) + if err != nil { + b.err = errors.Wrap(err, "get tombstones") + return false + } + + // NOTE: + // * block time range is half-open: [meta.MinTime, meta.MaxTime). + // * chunks are both closed: [chk.MinTime, chk.MaxTime]. + // * requested time ranges are closed: [req.Start, req.End]. + + var trimFront, trimBack bool + + // Copy chunks as iterables are reusable. + chks := make([]chunks.Meta, 0, len(b.bufChks)) + + // Prefilter chunks and pick those which are not entirely deleted or totally outside of the requested range. + for _, chk := range b.bufChks { + if chk.MaxTime < b.mint { + continue + } + if chk.MinTime > b.maxt { + continue + } + + if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(intervals)) { + chks = append(chks, chk) + } + + // If still not entirely deleted, check if trim is needed based on requested time range. + if !b.disableTrimming { + if chk.MinTime < b.mint { + trimFront = true + } + if chk.MaxTime > b.maxt { + trimBack = true + } + } + } + + if len(chks) == 0 { + continue + } + + if trimFront { + intervals = intervals.Add(tombstones.Interval{Mint: math.MinInt64, Maxt: b.mint - 1}) + } + if trimBack { + intervals = intervals.Add(tombstones.Interval{Mint: b.maxt + 1, Maxt: math.MaxInt64}) + } + + b.currLabels = make(labels.Labels, len(b.bufLbls)) + copy(b.currLabels, b.bufLbls) + + b.currIterFn = func() *populateWithDelGenericSeriesIterator { + return newPopulateWithDelGenericSeriesIterator(b.chunks, chks, intervals) + } + return true + } + return false +} + +func (b *blockBaseSeriesSet) Err() error { + if b.err != nil { + return b.err + } + return b.p.Err() +} + +func (b *blockBaseSeriesSet) Warnings() storage.Warnings { return nil } + +// populateWithDelGenericSeriesIterator allows to iterate over given chunk metas. In each iteration it ensures +// that chunks are trimmed based on given tombstones interval if any. +// +// populateWithDelGenericSeriesIterator assumes that chunks that would be fully removed by intervals are filtered out in previous phase. +// +// On each iteration currChkMeta is available. If currDelIter is not nil, it means that chunk iterator in currChkMeta +// is invalid and chunk rewrite is needed, currDelIter should be used. +type populateWithDelGenericSeriesIterator struct { + chunks ChunkReader + // chks are expected to be sorted by minTime and should be related to the same, single series. + chks []chunks.Meta + + i int + err error + bufIter *DeletedIterator + intervals tombstones.Intervals + + currDelIter chunkenc.Iterator + currChkMeta chunks.Meta +} + +func newPopulateWithDelGenericSeriesIterator( + chunks ChunkReader, + chks []chunks.Meta, + intervals tombstones.Intervals, +) *populateWithDelGenericSeriesIterator { + return &populateWithDelGenericSeriesIterator{ + chunks: chunks, + chks: chks, + i: -1, + bufIter: &DeletedIterator{}, + intervals: intervals, + } +} + +func (p *populateWithDelGenericSeriesIterator) next() bool { + if p.err != nil || p.i >= len(p.chks)-1 { + return false + } + + p.i++ + p.currChkMeta = p.chks[p.i] + + p.currChkMeta.Chunk, p.err = p.chunks.Chunk(p.currChkMeta.Ref) + if p.err != nil { + p.err = errors.Wrapf(p.err, "cannot populate chunk %d", p.currChkMeta.Ref) + return false + } + + p.bufIter.Intervals = p.bufIter.Intervals[:0] + for _, interval := range p.intervals { + if p.currChkMeta.OverlapsClosedInterval(interval.Mint, interval.Maxt) { + p.bufIter.Intervals = p.bufIter.Intervals.Add(interval) + } + } + + // Re-encode head chunks that are still open (being appended to) or + // outside the compacted MaxTime range. + // The chunk.Bytes() method is not safe for open chunks hence the re-encoding. + // This happens when snapshotting the head block or just fetching chunks from TSDB. + // + // TODO think how to avoid the typecasting to verify when it is head block. + _, isSafeChunk := p.currChkMeta.Chunk.(*safeChunk) + if len(p.bufIter.Intervals) == 0 && !(isSafeChunk && p.currChkMeta.MaxTime == math.MaxInt64) { + // If there are no overlap with deletion intervals AND it's NOT an "open" head chunk, we can take chunk as it is. + p.currDelIter = nil + return true + } + + // We don't want full chunk or it's potentially still opened, take just part of it. + p.bufIter.Iter = p.currChkMeta.Chunk.Iterator(nil) + p.currDelIter = p.bufIter + return true +} + +func (p *populateWithDelGenericSeriesIterator) Err() error { return p.err } + +func (p *populateWithDelGenericSeriesIterator) toSeriesIterator() chunkenc.Iterator { + return &populateWithDelSeriesIterator{populateWithDelGenericSeriesIterator: p} +} + +func (p *populateWithDelGenericSeriesIterator) toChunkSeriesIterator() chunks.Iterator { + return &populateWithDelChunkSeriesIterator{populateWithDelGenericSeriesIterator: p} +} + +// populateWithDelSeriesIterator allows to iterate over samples for the single series. +type populateWithDelSeriesIterator struct { + *populateWithDelGenericSeriesIterator + + curr chunkenc.Iterator +} + +func (p *populateWithDelSeriesIterator) Next() bool { + if p.curr != nil && p.curr.Next() { + return true + } + + for p.next() { + if p.currDelIter != nil { + p.curr = p.currDelIter + } else { + p.curr = p.currChkMeta.Chunk.Iterator(nil) + } + if p.curr.Next() { + return true + } + } + return false +} + +func (p *populateWithDelSeriesIterator) Seek(t int64) bool { + if p.curr != nil && p.curr.Seek(t) { + return true + } + for p.Next() { + if p.curr.Seek(t) { + return true + } + } + return false +} + +func (p *populateWithDelSeriesIterator) At() (int64, float64) { return p.curr.At() } + +func (p *populateWithDelSeriesIterator) Err() error { + if err := p.populateWithDelGenericSeriesIterator.Err(); err != nil { + return err + } + if p.curr != nil { + return p.curr.Err() + } + return nil +} + +type populateWithDelChunkSeriesIterator struct { + *populateWithDelGenericSeriesIterator + + curr chunks.Meta +} + +func (p *populateWithDelChunkSeriesIterator) Next() bool { + if !p.next() { + return false + } + + p.curr = p.currChkMeta + if p.currDelIter == nil { + return true + } + + // Re-encode the chunk if iterator is provider. This means that it has some samples to be deleted or chunk is opened. + newChunk := chunkenc.NewXORChunk() + app, err := newChunk.Appender() + if err != nil { + p.err = err + return false + } + + if !p.currDelIter.Next() { + if err := p.currDelIter.Err(); err != nil { + p.err = errors.Wrap(err, "iterate chunk while re-encoding") + return false + } + + // Empty chunk, this should not happen, as we assume full deletions being filtered before this iterator. + p.err = errors.Wrap(err, "populateWithDelChunkSeriesIterator: unexpected empty chunk found while rewriting chunk") + return false + } + + t, v := p.currDelIter.At() + p.curr.MinTime = t + app.Append(t, v) + + for p.currDelIter.Next() { + t, v = p.currDelIter.At() + app.Append(t, v) + } + if err := p.currDelIter.Err(); err != nil { + p.err = errors.Wrap(err, "iterate chunk while re-encoding") + return false + } + + p.curr.Chunk = newChunk + p.curr.MaxTime = t + return true +} + +func (p *populateWithDelChunkSeriesIterator) At() chunks.Meta { return p.curr } + +// blockSeriesSet allows to iterate over sorted, populated series with applied tombstones. +// Series with all deleted chunks are still present as Series with no samples. +// Samples from chunks are also trimmed to requested min and max time. +type blockSeriesSet struct { + blockBaseSeriesSet +} + +func newBlockSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings, mint, maxt int64, disableTrimming bool) storage.SeriesSet { + return &blockSeriesSet{ + blockBaseSeriesSet{ + index: i, + chunks: c, + tombstones: t, + p: p, + mint: mint, + maxt: maxt, + disableTrimming: disableTrimming, + bufLbls: make(labels.Labels, 0, 10), + }, + } +} + +func (b *blockSeriesSet) At() storage.Series { + // At can be looped over before iterating, so save the current value locally. + currIterFn := b.currIterFn + return &storage.SeriesEntry{ + Lset: b.currLabels, + SampleIteratorFn: func() chunkenc.Iterator { + return currIterFn().toSeriesIterator() + }, + } +} + +// blockChunkSeriesSet allows to iterate over sorted, populated series with applied tombstones. +// Series with all deleted chunks are still present as Labelled iterator with no chunks. +// Chunks are also trimmed to requested [min and max] (keeping samples with min and max timestamps). +type blockChunkSeriesSet struct { + blockBaseSeriesSet +} + +func newBlockChunkSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings, mint, maxt int64, disableTrimming bool) storage.ChunkSeriesSet { + return &blockChunkSeriesSet{ + blockBaseSeriesSet{ + index: i, + chunks: c, + tombstones: t, + p: p, + mint: mint, + maxt: maxt, + disableTrimming: disableTrimming, + bufLbls: make(labels.Labels, 0, 10), + }, + } +} + +func (b *blockChunkSeriesSet) At() storage.ChunkSeries { + // At can be looped over before iterating, so save the current value locally. + currIterFn := b.currIterFn + return &storage.ChunkSeriesEntry{ + Lset: b.currLabels, + ChunkIteratorFn: func() chunks.Iterator { + return currIterFn().toChunkSeriesIterator() + }, + } +} + +// NewMergedStringIter returns string iterator that allows to merge symbols on demand and stream result. +func NewMergedStringIter(a, b index.StringIter) index.StringIter { + return &mergedStringIter{a: a, b: b, aok: a.Next(), bok: b.Next()} +} + +type mergedStringIter struct { + a index.StringIter + b index.StringIter + aok, bok bool + cur string +} + +func (m *mergedStringIter) Next() bool { + if (!m.aok && !m.bok) || (m.Err() != nil) { + return false + } + + if !m.aok { + m.cur = m.b.At() + m.bok = m.b.Next() + } else if !m.bok { + m.cur = m.a.At() + m.aok = m.a.Next() + } else if m.b.At() > m.a.At() { + m.cur = m.a.At() + m.aok = m.a.Next() + } else if m.a.At() > m.b.At() { + m.cur = m.b.At() + m.bok = m.b.Next() + } else { // Equal. + m.cur = m.b.At() + m.aok = m.a.Next() + m.bok = m.b.Next() + } + + return true +} +func (m mergedStringIter) At() string { return m.cur } +func (m mergedStringIter) Err() error { + if m.a.Err() != nil { + return m.a.Err() + } + return m.b.Err() +} + +// DeletedIterator wraps chunk Iterator and makes sure any deleted metrics are not returned. +type DeletedIterator struct { + // Iter is an Iterator to be wrapped. + Iter chunkenc.Iterator + // Intervals are the deletion intervals. + Intervals tombstones.Intervals +} + +func (it *DeletedIterator) At() (int64, float64) { + return it.Iter.At() +} + +func (it *DeletedIterator) Seek(t int64) bool { + if it.Iter.Err() != nil { + return false + } + if ok := it.Iter.Seek(t); !ok { + return false + } + + // Now double check if the entry falls into a deleted interval. + ts, _ := it.At() + for _, itv := range it.Intervals { + if ts < itv.Mint { + return true + } + + if ts > itv.Maxt { + it.Intervals = it.Intervals[1:] + continue + } + + // We're in the middle of an interval, we can now call Next(). + return it.Next() + } + + // The timestamp is greater than all the deleted intervals. + return true +} + +func (it *DeletedIterator) Next() bool { +Outer: + for it.Iter.Next() { + ts, _ := it.Iter.At() + + for _, tr := range it.Intervals { + if tr.InBounds(ts) { + continue Outer + } + + if ts <= tr.Maxt { + return true + } + it.Intervals = it.Intervals[1:] + } + return true + } + return false +} + +func (it *DeletedIterator) Err() error { return it.Iter.Err() } + +type nopChunkReader struct { + emptyChunk chunkenc.Chunk +} + +func newNopChunkReader() ChunkReader { + return nopChunkReader{ + emptyChunk: chunkenc.NewXORChunk(), + } +} + +func (cr nopChunkReader) Chunk(ref chunks.ChunkRef) (chunkenc.Chunk, error) { + return cr.emptyChunk, nil +} + +func (cr nopChunkReader) Close() error { return nil } diff --git a/vendor/github.com/prometheus/prometheus/tsdb/repair.go b/vendor/github.com/prometheus/prometheus/tsdb/repair.go new file mode 100644 index 00000000000..8d60cb38177 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/repair.go @@ -0,0 +1,131 @@ +// Copyright 2018 The Prometheus Authors +// 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 tsdb + +import ( + "encoding/json" + "io" + "io/ioutil" + "os" + "path/filepath" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/fileutil" +) + +// repairBadIndexVersion repairs an issue in index and meta.json persistence introduced in +// commit 129773b41a565fde5156301e37f9a87158030443. +func repairBadIndexVersion(logger log.Logger, dir string) error { + // All blocks written by Prometheus 2.1 with a meta.json version of 2 are affected. + // We must actually set the index file version to 2 and revert the meta.json version back to 1. + dirs, err := blockDirs(dir) + if err != nil { + return errors.Wrapf(err, "list block dirs in %q", dir) + } + + tmpFiles := make([]string, 0, len(dirs)) + defer func() { + for _, tmp := range tmpFiles { + if err := os.RemoveAll(tmp); err != nil { + level.Error(logger).Log("msg", "remove tmp file", "err", err.Error()) + } + } + }() + + for _, d := range dirs { + meta, err := readBogusMetaFile(d) + if err != nil { + level.Error(logger).Log("msg", "failed to read meta.json for a block during repair process; skipping", "dir", d, "err", err) + continue + } + if meta.Version == metaVersion1 { + level.Info(logger).Log( + "msg", "Found healthy block", + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "ulid", meta.ULID, + ) + continue + } + level.Info(logger).Log( + "msg", "Fixing broken block", + "mint", meta.MinTime, + "maxt", meta.MaxTime, + "ulid", meta.ULID, + ) + + repl, err := os.Create(filepath.Join(d, "index.repaired")) + if err != nil { + return errors.Wrapf(err, "create index.repaired for block dir: %v", d) + } + tmpFiles = append(tmpFiles, repl.Name()) + + broken, err := os.Open(filepath.Join(d, indexFilename)) + if err != nil { + return errors.Wrapf(err, "open broken index for block dir: %v", d) + } + if _, err := io.Copy(repl, broken); err != nil { + return errors.Wrapf(err, "copy content of index to index.repaired for block dir: %v", d) + } + + // Set the 5th byte to 2 to indicate the correct file format version. + if _, err := repl.WriteAt([]byte{2}, 4); err != nil { + return tsdb_errors.NewMulti( + errors.Wrapf(err, "rewrite of index.repaired for block dir: %v", d), + errors.Wrap(repl.Close(), "close"), + ).Err() + } + if err := repl.Sync(); err != nil { + return tsdb_errors.NewMulti( + errors.Wrapf(err, "sync of index.repaired for block dir: %v", d), + errors.Wrap(repl.Close(), "close"), + ).Err() + } + if err := repl.Close(); err != nil { + return errors.Wrapf(repl.Close(), "close repaired index for block dir: %v", d) + } + if err := broken.Close(); err != nil { + return errors.Wrapf(repl.Close(), "close broken index for block dir: %v", d) + } + if err := fileutil.Replace(repl.Name(), broken.Name()); err != nil { + return errors.Wrapf(repl.Close(), "replaced broken index with index.repaired for block dir: %v", d) + } + // Reset version of meta.json to 1. + meta.Version = metaVersion1 + if _, err := writeMetaFile(logger, d, meta); err != nil { + return errors.Wrapf(repl.Close(), "write meta for block dir: %v", d) + } + } + return nil +} + +func readBogusMetaFile(dir string) (*BlockMeta, error) { + b, err := ioutil.ReadFile(filepath.Join(dir, metaFilename)) + if err != nil { + return nil, err + } + var m BlockMeta + + if err := json.Unmarshal(b, &m); err != nil { + return nil, err + } + if m.Version != metaVersion1 && m.Version != 2 { + return nil, errors.Errorf("unexpected meta file version %d", m.Version) + } + return &m, nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go b/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go new file mode 100644 index 00000000000..73244632472 --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/tsdbblockutil.go @@ -0,0 +1,76 @@ +// Copyright 2019 The Prometheus Authors +// 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 tsdb + +import ( + "context" + "fmt" + "path/filepath" + + "github.com/go-kit/log" + + "github.com/prometheus/prometheus/storage" +) + +var ErrInvalidTimes = fmt.Errorf("max time is lesser than min time") + +// CreateBlock creates a chunkrange block from the samples passed to it, and writes it to disk. +func CreateBlock(series []storage.Series, dir string, chunkRange int64, logger log.Logger) (string, error) { + if chunkRange == 0 { + chunkRange = DefaultBlockDuration + } + if chunkRange < 0 { + return "", ErrInvalidTimes + } + + w, err := NewBlockWriter(logger, dir, chunkRange) + if err != nil { + return "", err + } + defer func() { + if err := w.Close(); err != nil { + logger.Log("err closing blockwriter", err.Error()) + } + }() + + ctx := context.Background() + app := w.Appender(ctx) + + for _, s := range series { + ref := storage.SeriesRef(0) + it := s.Iterator() + lset := s.Labels() + for it.Next() { + t, v := it.At() + ref, err = app.Append(ref, lset, t, v) + if err != nil { + return "", err + } + } + if it.Err() != nil { + return "", it.Err() + } + } + + if err = app.Commit(); err != nil { + return "", err + } + + ulid, err := w.Flush(ctx) + if err != nil { + return "", err + } + + return filepath.Join(dir, ulid.String()), nil +} diff --git a/vendor/github.com/prometheus/prometheus/tsdb/wal.go b/vendor/github.com/prometheus/prometheus/tsdb/wal.go new file mode 100644 index 00000000000..2b9f117aa5e --- /dev/null +++ b/vendor/github.com/prometheus/prometheus/tsdb/wal.go @@ -0,0 +1,1309 @@ +// Copyright 2017 The Prometheus Authors +// 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 tsdb + +import ( + "bufio" + "encoding/binary" + "fmt" + "hash" + "hash/crc32" + "io" + "math" + "os" + "path/filepath" + "sort" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/encoding" + "github.com/prometheus/prometheus/tsdb/fileutil" + "github.com/prometheus/prometheus/tsdb/record" + "github.com/prometheus/prometheus/tsdb/tombstones" + "github.com/prometheus/prometheus/tsdb/wal" +) + +// WALEntryType indicates what data a WAL entry contains. +type WALEntryType uint8 + +const ( + // WALMagic is a 4 byte number every WAL segment file starts with. + WALMagic = uint32(0x43AF00EF) + + // WALFormatDefault is the version flag for the default outer segment file format. + WALFormatDefault = byte(1) +) + +// Entry types in a segment file. +const ( + WALEntrySymbols WALEntryType = 1 + WALEntrySeries WALEntryType = 2 + WALEntrySamples WALEntryType = 3 + WALEntryDeletes WALEntryType = 4 +) + +type walMetrics struct { + fsyncDuration prometheus.Summary + corruptions prometheus.Counter +} + +func newWalMetrics(r prometheus.Registerer) *walMetrics { + m := &walMetrics{} + + m.fsyncDuration = prometheus.NewSummary(prometheus.SummaryOpts{ + Name: "prometheus_tsdb_wal_fsync_duration_seconds", + Help: "Duration of WAL fsync.", + Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001}, + }) + m.corruptions = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "prometheus_tsdb_wal_corruptions_total", + Help: "Total number of WAL corruptions.", + }) + + if r != nil { + r.MustRegister( + m.fsyncDuration, + m.corruptions, + ) + } + return m +} + +// WAL is a write ahead log that can log new series labels and samples. +// It must be completely read before new entries are logged. +// +// DEPRECATED: use wal pkg combined with the record codex instead. +type WAL interface { + Reader() WALReader + LogSeries([]record.RefSeries) error + LogSamples([]record.RefSample) error + LogDeletes([]tombstones.Stone) error + Truncate(mint int64, keep func(uint64) bool) error + Close() error +} + +// WALReader reads entries from a WAL. +type WALReader interface { + Read( + seriesf func([]record.RefSeries), + samplesf func([]record.RefSample), + deletesf func([]tombstones.Stone), + ) error +} + +// segmentFile wraps a file object of a segment and tracks the highest timestamp +// it contains. During WAL truncating, all segments with no higher timestamp than +// the truncation threshold can be compacted. +type segmentFile struct { + *os.File + maxTime int64 // highest tombstone or sample timestamp in segment + minSeries chunks.HeadSeriesRef // lowerst series ID in segment +} + +func newSegmentFile(f *os.File) *segmentFile { + return &segmentFile{ + File: f, + maxTime: math.MinInt64, + minSeries: math.MaxUint64, + } +} + +const ( + walSegmentSizeBytes = 256 * 1024 * 1024 // 256 MB +) + +// The table gets initialized with sync.Once but may still cause a race +// with any other use of the crc32 package anywhere. Thus we initialize it +// before. +var castagnoliTable *crc32.Table + +func init() { + castagnoliTable = crc32.MakeTable(crc32.Castagnoli) +} + +// newCRC32 initializes a CRC32 hash with a preconfigured polynomial, so the +// polynomial may be easily changed in one location at a later time, if necessary. +func newCRC32() hash.Hash32 { + return crc32.New(castagnoliTable) +} + +// SegmentWAL is a write ahead log for series data. +// +// DEPRECATED: use wal pkg combined with the record coders instead. +type SegmentWAL struct { + mtx sync.Mutex + metrics *walMetrics + + dirFile *os.File + files []*segmentFile + + logger log.Logger + flushInterval time.Duration + segmentSize int64 + + crc32 hash.Hash32 + cur *bufio.Writer + curN int64 + + stopc chan struct{} + donec chan struct{} + actorc chan func() error // sequentialized background operations + buffers sync.Pool +} + +// OpenSegmentWAL opens or creates a write ahead log in the given directory. +// The WAL must be read completely before new data is written. +func OpenSegmentWAL(dir string, logger log.Logger, flushInterval time.Duration, r prometheus.Registerer) (*SegmentWAL, error) { + if err := os.MkdirAll(dir, 0o777); err != nil { + return nil, err + } + df, err := fileutil.OpenDir(dir) + if err != nil { + return nil, err + } + if logger == nil { + logger = log.NewNopLogger() + } + + w := &SegmentWAL{ + dirFile: df, + logger: logger, + flushInterval: flushInterval, + donec: make(chan struct{}), + stopc: make(chan struct{}), + actorc: make(chan func() error, 2), + segmentSize: walSegmentSizeBytes, + crc32: newCRC32(), + } + w.metrics = newWalMetrics(r) + + fns, err := sequenceFiles(w.dirFile.Name()) + if err != nil { + return nil, err + } + + for i, fn := range fns { + f, err := w.openSegmentFile(fn) + if err == nil { + w.files = append(w.files, newSegmentFile(f)) + continue + } + level.Warn(logger).Log("msg", "Invalid segment file detected, truncating WAL", "err", err, "file", fn) + + for _, fn := range fns[i:] { + if err := os.Remove(fn); err != nil { + return w, errors.Wrap(err, "removing segment failed") + } + } + break + } + + go w.run(flushInterval) + + return w, nil +} + +// repairingWALReader wraps a WAL reader and truncates its underlying SegmentWAL after the last +// valid entry if it encounters corruption. +type repairingWALReader struct { + wal *SegmentWAL + r WALReader +} + +func (r *repairingWALReader) Read( + seriesf func([]record.RefSeries), + samplesf func([]record.RefSample), + deletesf func([]tombstones.Stone), +) error { + err := r.r.Read(seriesf, samplesf, deletesf) + if err == nil { + return nil + } + cerr, ok := errors.Cause(err).(walCorruptionErr) + if !ok { + return err + } + r.wal.metrics.corruptions.Inc() + return r.wal.truncate(cerr.err, cerr.file, cerr.lastOffset) +} + +// truncate the WAL after the last valid entry. +func (w *SegmentWAL) truncate(err error, file int, lastOffset int64) error { + level.Error(w.logger).Log("msg", "WAL corruption detected; truncating", + "err", err, "file", w.files[file].Name(), "pos", lastOffset) + + // Close and delete all files after the current one. + for _, f := range w.files[file+1:] { + if err := f.Close(); err != nil { + return err + } + if err := os.Remove(f.Name()); err != nil { + return err + } + } + w.mtx.Lock() + defer w.mtx.Unlock() + + w.files = w.files[:file+1] + + // Seek the current file to the last valid offset where we continue writing from. + _, err = w.files[file].Seek(lastOffset, io.SeekStart) + return err +} + +// Reader returns a new reader over the write ahead log data. +// It must be completely consumed before writing to the WAL. +func (w *SegmentWAL) Reader() WALReader { + return &repairingWALReader{ + wal: w, + r: newWALReader(w.files, w.logger), + } +} + +func (w *SegmentWAL) getBuffer() *encoding.Encbuf { + b := w.buffers.Get() + if b == nil { + return &encoding.Encbuf{B: make([]byte, 0, 64*1024)} + } + return b.(*encoding.Encbuf) +} + +func (w *SegmentWAL) putBuffer(b *encoding.Encbuf) { + b.Reset() + w.buffers.Put(b) +} + +// Truncate deletes the values prior to mint and the series which the keep function +// does not indicate to preserve. +func (w *SegmentWAL) Truncate(mint int64, keep func(chunks.HeadSeriesRef) bool) error { + // The last segment is always active. + if len(w.files) < 2 { + return nil + } + var candidates []*segmentFile + + // All files have to be traversed as there could be two segments for a block + // with first block having times (10000, 20000) and SECOND one having (0, 10000). + for _, sf := range w.files[:len(w.files)-1] { + if sf.maxTime >= mint { + break + } + // Past WAL files are closed. We have to reopen them for another read. + f, err := w.openSegmentFile(sf.Name()) + if err != nil { + return errors.Wrap(err, "open old WAL segment for read") + } + candidates = append(candidates, &segmentFile{ + File: f, + minSeries: sf.minSeries, + maxTime: sf.maxTime, + }) + } + if len(candidates) == 0 { + return nil + } + + r := newWALReader(candidates, w.logger) + + // Create a new tmp file. + f, err := w.createSegmentFile(filepath.Join(w.dirFile.Name(), "compact.tmp")) + if err != nil { + return errors.Wrap(err, "create compaction segment") + } + defer func() { + if err := os.RemoveAll(f.Name()); err != nil { + level.Error(w.logger).Log("msg", "remove tmp file", "err", err.Error()) + } + }() + + var ( + csf = newSegmentFile(f) + crc32 = newCRC32() + decSeries = []record.RefSeries{} + activeSeries = []record.RefSeries{} + ) + + for r.next() { + rt, flag, byt := r.at() + + if rt != WALEntrySeries { + continue + } + decSeries = decSeries[:0] + activeSeries = activeSeries[:0] + + err := r.decodeSeries(flag, byt, &decSeries) + if err != nil { + return errors.Wrap(err, "decode samples while truncating") + } + for _, s := range decSeries { + if keep(s.Ref) { + activeSeries = append(activeSeries, s) + } + } + + buf := w.getBuffer() + flag = w.encodeSeries(buf, activeSeries) + + _, err = w.writeTo(csf, crc32, WALEntrySeries, flag, buf.Get()) + w.putBuffer(buf) + + if err != nil { + return errors.Wrap(err, "write to compaction segment") + } + } + if r.Err() != nil { + return errors.Wrap(r.Err(), "read candidate WAL files") + } + + off, err := csf.Seek(0, io.SeekCurrent) + if err != nil { + return err + } + if err := csf.Truncate(off); err != nil { + return err + } + if err := csf.Sync(); err != nil { + return nil + } + if err := csf.Close(); err != nil { + return nil + } + + _ = candidates[0].Close() // need close before remove on platform windows + if err := fileutil.Replace(csf.Name(), candidates[0].Name()); err != nil { + return errors.Wrap(err, "rename compaction segment") + } + for _, f := range candidates[1:] { + f.Close() // need close before remove on platform windows + if err := os.RemoveAll(f.Name()); err != nil { + return errors.Wrap(err, "delete WAL segment file") + } + } + if err := w.dirFile.Sync(); err != nil { + return err + } + + // The file object of csf still holds the name before rename. Recreate it so + // subsequent truncations do not look at a non-existent file name. + csf.File, err = w.openSegmentFile(candidates[0].Name()) + if err != nil { + return err + } + // We don't need it to be open. + if err := csf.Close(); err != nil { + return err + } + + w.mtx.Lock() + w.files = append([]*segmentFile{csf}, w.files[len(candidates):]...) + w.mtx.Unlock() + + return nil +} + +// LogSeries writes a batch of new series labels to the log. +// The series have to be ordered. +func (w *SegmentWAL) LogSeries(series []record.RefSeries) error { + buf := w.getBuffer() + + flag := w.encodeSeries(buf, series) + + w.mtx.Lock() + defer w.mtx.Unlock() + + err := w.write(WALEntrySeries, flag, buf.Get()) + + w.putBuffer(buf) + + if err != nil { + return errors.Wrap(err, "log series") + } + + tf := w.head() + + for _, s := range series { + if tf.minSeries > s.Ref { + tf.minSeries = s.Ref + } + } + return nil +} + +// LogSamples writes a batch of new samples to the log. +func (w *SegmentWAL) LogSamples(samples []record.RefSample) error { + buf := w.getBuffer() + + flag := w.encodeSamples(buf, samples) + + w.mtx.Lock() + defer w.mtx.Unlock() + + err := w.write(WALEntrySamples, flag, buf.Get()) + + w.putBuffer(buf) + + if err != nil { + return errors.Wrap(err, "log series") + } + tf := w.head() + + for _, s := range samples { + if tf.maxTime < s.T { + tf.maxTime = s.T + } + } + return nil +} + +// LogDeletes write a batch of new deletes to the log. +func (w *SegmentWAL) LogDeletes(stones []tombstones.Stone) error { + buf := w.getBuffer() + + flag := w.encodeDeletes(buf, stones) + + w.mtx.Lock() + defer w.mtx.Unlock() + + err := w.write(WALEntryDeletes, flag, buf.Get()) + + w.putBuffer(buf) + + if err != nil { + return errors.Wrap(err, "log series") + } + tf := w.head() + + for _, s := range stones { + for _, iv := range s.Intervals { + if tf.maxTime < iv.Maxt { + tf.maxTime = iv.Maxt + } + } + } + return nil +} + +// openSegmentFile opens the given segment file and consumes and validates header. +func (w *SegmentWAL) openSegmentFile(name string) (*os.File, error) { + // We must open all files in read/write mode as we may have to truncate along + // the way and any file may become the head. + f, err := os.OpenFile(name, os.O_RDWR, 0o666) + if err != nil { + return nil, err + } + metab := make([]byte, 8) + + // If there is an error, we need close f for platform windows before gc. + // Otherwise, file op may fail. + hasError := true + defer func() { + if hasError { + f.Close() + } + }() + + if n, err := f.Read(metab); err != nil { + return nil, errors.Wrapf(err, "validate meta %q", f.Name()) + } else if n != 8 { + return nil, errors.Errorf("invalid header size %d in %q", n, f.Name()) + } + + if m := binary.BigEndian.Uint32(metab[:4]); m != WALMagic { + return nil, errors.Errorf("invalid magic header %x in %q", m, f.Name()) + } + if metab[4] != WALFormatDefault { + return nil, errors.Errorf("unknown WAL segment format %d in %q", metab[4], f.Name()) + } + hasError = false + return f, nil +} + +// createSegmentFile creates a new segment file with the given name. It preallocates +// the standard segment size if possible and writes the header. +func (w *SegmentWAL) createSegmentFile(name string) (*os.File, error) { + f, err := os.Create(name) + if err != nil { + return nil, err + } + if err = fileutil.Preallocate(f, w.segmentSize, true); err != nil { + return nil, err + } + // Write header metadata for new file. + metab := make([]byte, 8) + binary.BigEndian.PutUint32(metab[:4], WALMagic) + metab[4] = WALFormatDefault + + if _, err := f.Write(metab); err != nil { + return nil, err + } + return f, err +} + +// cut finishes the currently active segments and opens the next one. +// The encoder is reset to point to the new segment. +func (w *SegmentWAL) cut() error { + // Sync current head to disk and close. + if hf := w.head(); hf != nil { + if err := w.flush(); err != nil { + return err + } + // Finish last segment asynchronously to not block the WAL moving along + // in the new segment. + go func() { + w.actorc <- func() error { + off, err := hf.Seek(0, io.SeekCurrent) + if err != nil { + return errors.Wrapf(err, "finish old segment %s", hf.Name()) + } + if err := hf.Truncate(off); err != nil { + return errors.Wrapf(err, "finish old segment %s", hf.Name()) + } + if err := hf.Sync(); err != nil { + return errors.Wrapf(err, "finish old segment %s", hf.Name()) + } + if err := hf.Close(); err != nil { + return errors.Wrapf(err, "finish old segment %s", hf.Name()) + } + return nil + } + }() + } + + p, _, err := nextSequenceFile(w.dirFile.Name()) + if err != nil { + return err + } + f, err := w.createSegmentFile(p) + if err != nil { + return err + } + + go func() { + w.actorc <- func() error { + return errors.Wrap(w.dirFile.Sync(), "sync WAL directory") + } + }() + + w.files = append(w.files, newSegmentFile(f)) + + // TODO(gouthamve): make the buffer size a constant. + w.cur = bufio.NewWriterSize(f, 8*1024*1024) + w.curN = 8 + + return nil +} + +func (w *SegmentWAL) head() *segmentFile { + if len(w.files) == 0 { + return nil + } + return w.files[len(w.files)-1] +} + +// Sync flushes the changes to disk. +func (w *SegmentWAL) Sync() error { + var head *segmentFile + var err error + + // Flush the writer and retrieve the reference to the head segment under mutex lock. + func() { + w.mtx.Lock() + defer w.mtx.Unlock() + if err = w.flush(); err != nil { + return + } + head = w.head() + }() + if err != nil { + return errors.Wrap(err, "flush buffer") + } + if head != nil { + // But only fsync the head segment after releasing the mutex as it will block on disk I/O. + start := time.Now() + err := fileutil.Fdatasync(head.File) + w.metrics.fsyncDuration.Observe(time.Since(start).Seconds()) + return err + } + return nil +} + +func (w *SegmentWAL) sync() error { + if err := w.flush(); err != nil { + return err + } + if w.head() == nil { + return nil + } + + start := time.Now() + err := fileutil.Fdatasync(w.head().File) + w.metrics.fsyncDuration.Observe(time.Since(start).Seconds()) + return err +} + +func (w *SegmentWAL) flush() error { + if w.cur == nil { + return nil + } + return w.cur.Flush() +} + +func (w *SegmentWAL) run(interval time.Duration) { + var tick <-chan time.Time + + if interval > 0 { + ticker := time.NewTicker(interval) + defer ticker.Stop() + tick = ticker.C + } + defer close(w.donec) + + for { + // Processing all enqueued operations has precedence over shutdown and + // background syncs. + select { + case f := <-w.actorc: + if err := f(); err != nil { + level.Error(w.logger).Log("msg", "operation failed", "err", err) + } + continue + default: + } + select { + case <-w.stopc: + return + case f := <-w.actorc: + if err := f(); err != nil { + level.Error(w.logger).Log("msg", "operation failed", "err", err) + } + case <-tick: + if err := w.Sync(); err != nil { + level.Error(w.logger).Log("msg", "sync failed", "err", err) + } + } + } +} + +// Close syncs all data and closes the underlying resources. +func (w *SegmentWAL) Close() error { + // Make sure you can call Close() multiple times. + select { + case <-w.stopc: + return nil // Already closed. + default: + } + + close(w.stopc) + <-w.donec + + w.mtx.Lock() + defer w.mtx.Unlock() + + if err := w.sync(); err != nil { + return err + } + // On opening, a WAL must be fully consumed once. Afterwards + // only the current segment will still be open. + if hf := w.head(); hf != nil { + if err := hf.Close(); err != nil { + return errors.Wrapf(err, "closing WAL head %s", hf.Name()) + } + } + + return errors.Wrapf(w.dirFile.Close(), "closing WAL dir %s", w.dirFile.Name()) +} + +func (w *SegmentWAL) write(t WALEntryType, flag uint8, buf []byte) error { + // Cut to the next segment if the entry exceeds the file size unless it would also + // exceed the size of a new segment. + // TODO(gouthamve): Add a test for this case where the commit is greater than segmentSize. + var ( + sz = int64(len(buf)) + 6 + newsz = w.curN + sz + ) + // XXX(fabxc): this currently cuts a new file whenever the WAL was newly opened. + // Probably fine in general but may yield a lot of short files in some cases. + if w.cur == nil || w.curN > w.segmentSize || newsz > w.segmentSize && sz <= w.segmentSize { + if err := w.cut(); err != nil { + return err + } + } + n, err := w.writeTo(w.cur, w.crc32, t, flag, buf) + + w.curN += int64(n) + + return err +} + +func (w *SegmentWAL) writeTo(wr io.Writer, crc32 hash.Hash, t WALEntryType, flag uint8, buf []byte) (int, error) { + if len(buf) == 0 { + return 0, nil + } + crc32.Reset() + wr = io.MultiWriter(crc32, wr) + + var b [6]byte + b[0] = byte(t) + b[1] = flag + + binary.BigEndian.PutUint32(b[2:], uint32(len(buf))) + + n1, err := wr.Write(b[:]) + if err != nil { + return n1, err + } + n2, err := wr.Write(buf) + if err != nil { + return n1 + n2, err + } + n3, err := wr.Write(crc32.Sum(b[:0])) + + return n1 + n2 + n3, err +} + +const ( + walSeriesSimple = 1 + walSamplesSimple = 1 + walDeletesSimple = 1 +) + +func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []record.RefSeries) uint8 { + for _, s := range series { + buf.PutBE64(uint64(s.Ref)) + buf.PutUvarint(len(s.Labels)) + + for _, l := range s.Labels { + buf.PutUvarintStr(l.Name) + buf.PutUvarintStr(l.Value) + } + } + return walSeriesSimple +} + +func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []record.RefSample) uint8 { + if len(samples) == 0 { + return walSamplesSimple + } + // Store base timestamp and base reference number of first sample. + // All samples encode their timestamp and ref as delta to those. + // + // TODO(fabxc): optimize for all samples having the same timestamp. + first := samples[0] + + buf.PutBE64(uint64(first.Ref)) + buf.PutBE64int64(first.T) + + for _, s := range samples { + buf.PutVarint64(int64(s.Ref) - int64(first.Ref)) + buf.PutVarint64(s.T - first.T) + buf.PutBE64(math.Float64bits(s.V)) + } + return walSamplesSimple +} + +func (w *SegmentWAL) encodeDeletes(buf *encoding.Encbuf, stones []tombstones.Stone) uint8 { + for _, s := range stones { + for _, iv := range s.Intervals { + buf.PutBE64(uint64(s.Ref)) + buf.PutVarint64(iv.Mint) + buf.PutVarint64(iv.Maxt) + } + } + return walDeletesSimple +} + +// walReader decodes and emits write ahead log entries. +type walReader struct { + logger log.Logger + + files []*segmentFile + cur int + buf []byte + crc32 hash.Hash32 + + curType WALEntryType + curFlag byte + curBuf []byte + lastOffset int64 // offset after last successfully read entry + + err error +} + +func newWALReader(files []*segmentFile, l log.Logger) *walReader { + if l == nil { + l = log.NewNopLogger() + } + return &walReader{ + logger: l, + files: files, + buf: make([]byte, 0, 128*4096), + crc32: newCRC32(), + } +} + +// Err returns the last error the reader encountered. +func (r *walReader) Err() error { + return r.err +} + +func (r *walReader) Read( + seriesf func([]record.RefSeries), + samplesf func([]record.RefSample), + deletesf func([]tombstones.Stone), +) error { + // Concurrency for replaying the WAL is very limited. We at least split out decoding and + // processing into separate threads. + // Historically, the processing is the bottleneck with reading and decoding using only + // 15% of the CPU. + var ( + seriesPool sync.Pool + samplePool sync.Pool + deletePool sync.Pool + ) + donec := make(chan struct{}) + datac := make(chan interface{}, 100) + + go func() { + defer close(donec) + + for x := range datac { + switch v := x.(type) { + case []record.RefSeries: + if seriesf != nil { + seriesf(v) + } + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + seriesPool.Put(v[:0]) + case []record.RefSample: + if samplesf != nil { + samplesf(v) + } + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + samplePool.Put(v[:0]) + case []tombstones.Stone: + if deletesf != nil { + deletesf(v) + } + //nolint:staticcheck // Ignore SA6002 safe to ignore and actually fixing it has some performance penalty. + deletePool.Put(v[:0]) + default: + level.Error(r.logger).Log("msg", "unexpected data type") + } + } + }() + + var err error + + for r.next() { + et, flag, b := r.at() + + // In decoding below we never return a walCorruptionErr for now. + // Those should generally be caught by entry decoding before. + switch et { + case WALEntrySeries: + var series []record.RefSeries + if v := seriesPool.Get(); v == nil { + series = make([]record.RefSeries, 0, 512) + } else { + series = v.([]record.RefSeries) + } + + err = r.decodeSeries(flag, b, &series) + if err != nil { + err = errors.Wrap(err, "decode series entry") + break + } + datac <- series + + cf := r.current() + for _, s := range series { + if cf.minSeries > s.Ref { + cf.minSeries = s.Ref + } + } + case WALEntrySamples: + var samples []record.RefSample + if v := samplePool.Get(); v == nil { + samples = make([]record.RefSample, 0, 512) + } else { + samples = v.([]record.RefSample) + } + + err = r.decodeSamples(flag, b, &samples) + if err != nil { + err = errors.Wrap(err, "decode samples entry") + break + } + datac <- samples + + // Update the times for the WAL segment file. + cf := r.current() + for _, s := range samples { + if cf.maxTime < s.T { + cf.maxTime = s.T + } + } + case WALEntryDeletes: + var deletes []tombstones.Stone + if v := deletePool.Get(); v == nil { + deletes = make([]tombstones.Stone, 0, 512) + } else { + deletes = v.([]tombstones.Stone) + } + + err = r.decodeDeletes(flag, b, &deletes) + if err != nil { + err = errors.Wrap(err, "decode delete entry") + break + } + datac <- deletes + + // Update the times for the WAL segment file. + cf := r.current() + for _, s := range deletes { + for _, iv := range s.Intervals { + if cf.maxTime < iv.Maxt { + cf.maxTime = iv.Maxt + } + } + } + } + } + close(datac) + <-donec + + if err != nil { + return err + } + if r.Err() != nil { + return errors.Wrap(r.Err(), "read entry") + } + return nil +} + +func (r *walReader) at() (WALEntryType, byte, []byte) { + return r.curType, r.curFlag, r.curBuf +} + +// next returns decodes the next entry pair and returns true +// if it was successful. +func (r *walReader) next() bool { + if r.cur >= len(r.files) { + return false + } + cf := r.files[r.cur] + + // Remember the offset after the last correctly read entry. If the next one + // is corrupted, this is where we can safely truncate. + r.lastOffset, r.err = cf.Seek(0, io.SeekCurrent) + if r.err != nil { + return false + } + + et, flag, b, err := r.entry(cf) + // If we reached the end of the reader, advance to the next one + // and close. + // Do not close on the last one as it will still be appended to. + if err == io.EOF { + if r.cur == len(r.files)-1 { + return false + } + // Current reader completed, close and move to the next one. + if err := cf.Close(); err != nil { + r.err = err + return false + } + r.cur++ + return r.next() + } + if err != nil { + r.err = err + return false + } + + r.curType = et + r.curFlag = flag + r.curBuf = b + return r.err == nil +} + +func (r *walReader) current() *segmentFile { + return r.files[r.cur] +} + +// walCorruptionErr is a type wrapper for errors that indicate WAL corruption +// and trigger a truncation. +type walCorruptionErr struct { + err error + file int + lastOffset int64 +} + +func (e walCorruptionErr) Error() string { + return fmt.Sprintf("%s ", e.err, e.file, e.lastOffset) +} + +func (r *walReader) corruptionErr(s string, args ...interface{}) error { + return walCorruptionErr{ + err: errors.Errorf(s, args...), + file: r.cur, + lastOffset: r.lastOffset, + } +} + +func (r *walReader) entry(cr io.Reader) (WALEntryType, byte, []byte, error) { + r.crc32.Reset() + tr := io.TeeReader(cr, r.crc32) + + b := make([]byte, 6) + if n, err := tr.Read(b); err != nil { + return 0, 0, nil, err + } else if n != 6 { + return 0, 0, nil, r.corruptionErr("invalid entry header size %d", n) + } + + var ( + etype = WALEntryType(b[0]) + flag = b[1] + length = int(binary.BigEndian.Uint32(b[2:])) + ) + // Exit if we reached pre-allocated space. + if etype == 0 { + return 0, 0, nil, io.EOF + } + if etype != WALEntrySeries && etype != WALEntrySamples && etype != WALEntryDeletes { + return 0, 0, nil, r.corruptionErr("invalid entry type %d", etype) + } + + if length > len(r.buf) { + r.buf = make([]byte, length) + } + buf := r.buf[:length] + + if n, err := tr.Read(buf); err != nil { + return 0, 0, nil, err + } else if n != length { + return 0, 0, nil, r.corruptionErr("invalid entry body size %d", n) + } + + if n, err := cr.Read(b[:4]); err != nil { + return 0, 0, nil, err + } else if n != 4 { + return 0, 0, nil, r.corruptionErr("invalid checksum length %d", n) + } + if exp, has := binary.BigEndian.Uint32(b[:4]), r.crc32.Sum32(); has != exp { + return 0, 0, nil, r.corruptionErr("unexpected CRC32 checksum %x, want %x", has, exp) + } + + return etype, flag, buf, nil +} + +func (r *walReader) decodeSeries(flag byte, b []byte, res *[]record.RefSeries) error { + dec := encoding.Decbuf{B: b} + + for len(dec.B) > 0 && dec.Err() == nil { + ref := chunks.HeadSeriesRef(dec.Be64()) + + lset := make(labels.Labels, dec.Uvarint()) + + for i := range lset { + lset[i].Name = dec.UvarintStr() + lset[i].Value = dec.UvarintStr() + } + sort.Sort(lset) + + *res = append(*res, record.RefSeries{ + Ref: ref, + Labels: lset, + }) + } + if dec.Err() != nil { + return dec.Err() + } + if len(dec.B) > 0 { + return errors.Errorf("unexpected %d bytes left in entry", len(dec.B)) + } + return nil +} + +func (r *walReader) decodeSamples(flag byte, b []byte, res *[]record.RefSample) error { + if len(b) == 0 { + return nil + } + dec := encoding.Decbuf{B: b} + + var ( + baseRef = dec.Be64() + baseTime = dec.Be64int64() + ) + + for len(dec.B) > 0 && dec.Err() == nil { + dref := dec.Varint64() + dtime := dec.Varint64() + val := dec.Be64() + + *res = append(*res, record.RefSample{ + Ref: chunks.HeadSeriesRef(int64(baseRef) + dref), + T: baseTime + dtime, + V: math.Float64frombits(val), + }) + } + + if dec.Err() != nil { + return errors.Wrapf(dec.Err(), "decode error after %d samples", len(*res)) + } + if len(dec.B) > 0 { + return errors.Errorf("unexpected %d bytes left in entry", len(dec.B)) + } + return nil +} + +func (r *walReader) decodeDeletes(flag byte, b []byte, res *[]tombstones.Stone) error { + dec := &encoding.Decbuf{B: b} + + for dec.Len() > 0 && dec.Err() == nil { + *res = append(*res, tombstones.Stone{ + Ref: storage.SeriesRef(dec.Be64()), + Intervals: tombstones.Intervals{ + {Mint: dec.Varint64(), Maxt: dec.Varint64()}, + }, + }) + } + if dec.Err() != nil { + return dec.Err() + } + if len(dec.B) > 0 { + return errors.Errorf("unexpected %d bytes left in entry", len(dec.B)) + } + return nil +} + +func deprecatedWALExists(logger log.Logger, dir string) (bool, error) { + // Detect whether we still have the old WAL. + fns, err := sequenceFiles(dir) + if err != nil && !os.IsNotExist(err) { + return false, errors.Wrap(err, "list sequence files") + } + if len(fns) == 0 { + return false, nil // No WAL at all yet. + } + // Check header of first segment to see whether we are still dealing with an + // old WAL. + f, err := os.Open(fns[0]) + if err != nil { + return false, errors.Wrap(err, "check first existing segment") + } + defer f.Close() + + var hdr [4]byte + if _, err := f.Read(hdr[:]); err != nil && err != io.EOF { + return false, errors.Wrap(err, "read header from first segment") + } + // If we cannot read the magic header for segments of the old WAL, abort. + // Either it's migrated already or there's a corruption issue with which + // we cannot deal here anyway. Subsequent attempts to open the WAL will error in that case. + if binary.BigEndian.Uint32(hdr[:]) != WALMagic { + return false, nil + } + return true, nil +} + +// MigrateWAL rewrites the deprecated write ahead log into the new format. +func MigrateWAL(logger log.Logger, dir string) (err error) { + if logger == nil { + logger = log.NewNopLogger() + } + if exists, err := deprecatedWALExists(logger, dir); err != nil || !exists { + return err + } + level.Info(logger).Log("msg", "Migrating WAL format") + + tmpdir := dir + ".tmp" + if err := os.RemoveAll(tmpdir); err != nil { + return errors.Wrap(err, "cleanup replacement dir") + } + repl, err := wal.New(logger, nil, tmpdir, false) + if err != nil { + return errors.Wrap(err, "open new WAL") + } + + // It should've already been closed as part of the previous finalization. + // Do it once again in case of prior errors. + defer func() { + if err != nil { + repl.Close() + } + }() + + w, err := OpenSegmentWAL(dir, logger, time.Minute, nil) + if err != nil { + return errors.Wrap(err, "open old WAL") + } + defer w.Close() + + rdr := w.Reader() + + var ( + enc record.Encoder + b []byte + ) + decErr := rdr.Read( + func(s []record.RefSeries) { + if err != nil { + return + } + err = repl.Log(enc.Series(s, b[:0])) + }, + func(s []record.RefSample) { + if err != nil { + return + } + err = repl.Log(enc.Samples(s, b[:0])) + }, + func(s []tombstones.Stone) { + if err != nil { + return + } + err = repl.Log(enc.Tombstones(s, b[:0])) + }, + ) + if decErr != nil { + return errors.Wrap(err, "decode old entries") + } + if err != nil { + return errors.Wrap(err, "write new entries") + } + // We explicitly close even when there is a defer for Windows to be + // able to delete it. The defer is in place to close it in-case there + // are errors above. + if err := w.Close(); err != nil { + return errors.Wrap(err, "close old WAL") + } + if err := repl.Close(); err != nil { + return errors.Wrap(err, "close new WAL") + } + if err := fileutil.Replace(tmpdir, dir); err != nil { + return errors.Wrap(err, "replace old WAL") + } + return nil +} diff --git a/vendor/golang.org/x/sync/errgroup/errgroup.go b/vendor/golang.org/x/sync/errgroup/errgroup.go new file mode 100644 index 00000000000..9857fe53d3c --- /dev/null +++ b/vendor/golang.org/x/sync/errgroup/errgroup.go @@ -0,0 +1,66 @@ +// Copyright 2016 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +// Package errgroup provides synchronization, error propagation, and Context +// cancelation for groups of goroutines working on subtasks of a common task. +package errgroup + +import ( + "context" + "sync" +) + +// A Group is a collection of goroutines working on subtasks that are part of +// the same overall task. +// +// A zero Group is valid and does not cancel on error. +type Group struct { + cancel func() + + wg sync.WaitGroup + + errOnce sync.Once + err error +} + +// WithContext returns a new Group and an associated Context derived from ctx. +// +// The derived Context is canceled the first time a function passed to Go +// returns a non-nil error or the first time Wait returns, whichever occurs +// first. +func WithContext(ctx context.Context) (*Group, context.Context) { + ctx, cancel := context.WithCancel(ctx) + return &Group{cancel: cancel}, ctx +} + +// Wait blocks until all function calls from the Go method have returned, then +// returns the first non-nil error (if any) from them. +func (g *Group) Wait() error { + g.wg.Wait() + if g.cancel != nil { + g.cancel() + } + return g.err +} + +// Go calls the given function in a new goroutine. +// +// The first call to return a non-nil error cancels the group; its error will be +// returned by Wait. +func (g *Group) Go(f func() error) { + g.wg.Add(1) + + go func() { + defer g.wg.Done() + + if err := f(); err != nil { + g.errOnce.Do(func() { + g.err = err + if g.cancel != nil { + g.cancel() + } + }) + } + }() +} diff --git a/vendor/modules.txt b/vendor/modules.txt index 3dddf658df1..108188abf49 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -516,7 +516,6 @@ github.com/klauspost/compress/gzip github.com/klauspost/compress/huff0 github.com/klauspost/compress/internal/snapref github.com/klauspost/compress/s2 -github.com/klauspost/compress/snappy github.com/klauspost/compress/zstd github.com/klauspost/compress/zstd/internal/xxhash # github.com/klauspost/cpuid v1.3.1 @@ -601,6 +600,9 @@ github.com/mwitkow/go-conntrack # github.com/oklog/run v1.1.0 ## explicit; go 1.13 github.com/oklog/run +# github.com/oklog/ulid v1.3.1 +## explicit +github.com/oklog/ulid # github.com/olekukonko/tablewriter v0.0.2 ## explicit; go 1.12 github.com/olekukonko/tablewriter @@ -728,11 +730,14 @@ github.com/prometheus/prometheus/prompb github.com/prometheus/prometheus/scrape github.com/prometheus/prometheus/storage github.com/prometheus/prometheus/storage/remote +github.com/prometheus/prometheus/tsdb +github.com/prometheus/prometheus/tsdb/agent github.com/prometheus/prometheus/tsdb/chunkenc github.com/prometheus/prometheus/tsdb/chunks github.com/prometheus/prometheus/tsdb/encoding github.com/prometheus/prometheus/tsdb/errors github.com/prometheus/prometheus/tsdb/fileutil +github.com/prometheus/prometheus/tsdb/goversion github.com/prometheus/prometheus/tsdb/index github.com/prometheus/prometheus/tsdb/record github.com/prometheus/prometheus/tsdb/tombstones @@ -1107,6 +1112,7 @@ golang.org/x/oauth2/jws golang.org/x/oauth2/jwt # golang.org/x/sync v0.0.0-20210220032951-036812b2e83c ## explicit +golang.org/x/sync/errgroup golang.org/x/sync/semaphore # golang.org/x/sys v0.0.0-20220222172238-00053529121e ## explicit; go 1.17