otlp: Prometheus to own its own copy of the otlptranslator package (#13991)
After a lot of productive discussion between the Prometheus and OpenTelemetry community we decided that it made sense for Prometheus to own its own copy of the code in charge for handling OTLP ingestion traffic. This commit is removing the README and update-copy.sh files that had the previous steps to update the code. Also it is updating the licensing of all the files to make sure the OpenTelemetry provenance is explicit and to state the new ownership. Signed-off-by: Jesus Vazquez <jesusvzpg@gmail.com> Co-authored-by: Arve Knudsen <arve.knudsen@gmail.com>
This commit is contained in:
parent
9fda9443d4
commit
7554384dac
|
@ -1,22 +0,0 @@
|
|||
## Copying from opentelemetry/opentelemetry-collector-contrib
|
||||
|
||||
This files in the `prometheus/` and `prometheusremotewrite/` are copied from the OpenTelemetry Project[^1].
|
||||
|
||||
This is done instead of adding a go.mod dependency because OpenTelemetry depends on `prometheus/prometheus` and a cyclic dependency will be created. This is just a temporary solution and the long-term solution is to move the required packages from OpenTelemetry into `prometheus/prometheus`.
|
||||
|
||||
To update the dependency is a multi-step process:
|
||||
1. Vendor the latest `prometheus/prometheus`@`main` into [`opentelemetry/opentelemetry-collector-contrib`](https://github.com/open-telemetry/opentelemetry-collector-contrib)
|
||||
1. Update the VERSION in `update-copy.sh`.
|
||||
1. Run `./update-copy.sh`.
|
||||
|
||||
### Why copy?
|
||||
|
||||
This is because the packages we copy depend on the [`prompb`](https://github.com/prometheus/prometheus/blob/main/prompb) package. While the package is relatively stable, there are still changes. For example, https://github.com/prometheus/prometheus/pull/11935 changed the types.
|
||||
This means if we depend on the upstream packages directly, we will never able to make the changes like above. Hence we're copying the code for now.
|
||||
|
||||
### I need to manually change these files
|
||||
|
||||
When we do want to make changes to the types in `prompb`, we might need to edit the files directly. That is OK, please let @gouthamve or @jesusvazquez know so they can take care of updating the upstream code (by vendoring in `prometheus/prometheus` upstream and resolving conflicts) and then will run the copy
|
||||
script again to keep things updated.
|
||||
|
||||
[^1]: https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/pkg/translator/prometheus and https://github.com/open-telemetry/opentelemetry-collector-contrib/tree/main/pkg/translator/prometheusremotewrite
|
|
@ -1,9 +1,20 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheus/normalize_label.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheus // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
package prometheus
|
||||
|
||||
import (
|
||||
"strings"
|
||||
|
|
|
@ -1,9 +1,20 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheus/normalize_name.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheus // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
package prometheus
|
||||
|
||||
import (
|
||||
"strings"
|
||||
|
|
|
@ -1,9 +1,20 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheus/unit_to_ucum.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheus // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
package prometheus
|
||||
|
||||
import "strings"
|
||||
|
||||
|
|
|
@ -1,29 +1,42 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/helper.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheusremotewrite // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
|
||||
package prometheusremotewrite
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"log"
|
||||
"math"
|
||||
"slices"
|
||||
"sort"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
"unicode/utf8"
|
||||
|
||||
"github.com/cespare/xxhash/v2"
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/prometheus/prometheus/model/timestamp"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"go.opentelemetry.io/collector/pdata/pcommon"
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
conventions "go.opentelemetry.io/collector/semconv/v1.6.1"
|
||||
|
||||
"github.com/prometheus/prometheus/model/timestamp"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
|
||||
prometheustranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
)
|
||||
|
||||
|
@ -48,7 +61,7 @@ const (
|
|||
)
|
||||
|
||||
type bucketBoundsData struct {
|
||||
sig string
|
||||
ts *prompb.TimeSeries
|
||||
bound float64
|
||||
}
|
||||
|
||||
|
@ -66,94 +79,47 @@ func (a ByLabelName) Len() int { return len(a) }
|
|||
func (a ByLabelName) Less(i, j int) bool { return a[i].Name < a[j].Name }
|
||||
func (a ByLabelName) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
|
||||
|
||||
// addSample finds a TimeSeries in tsMap that corresponds to the label set labels, and add sample to the TimeSeries; it
|
||||
// creates a new TimeSeries in the map if not found and returns the time series signature.
|
||||
// tsMap will be unmodified if either labels or sample is nil, but can still be modified if the exemplar is nil.
|
||||
func addSample(tsMap map[string]*prompb.TimeSeries, sample *prompb.Sample, labels []prompb.Label,
|
||||
datatype string) string {
|
||||
if sample == nil || labels == nil || tsMap == nil {
|
||||
// This shouldn't happen
|
||||
return ""
|
||||
}
|
||||
|
||||
sig := timeSeriesSignature(datatype, labels)
|
||||
ts := tsMap[sig]
|
||||
if ts != nil {
|
||||
ts.Samples = append(ts.Samples, *sample)
|
||||
} else {
|
||||
newTs := &prompb.TimeSeries{
|
||||
Labels: labels,
|
||||
Samples: []prompb.Sample{*sample},
|
||||
}
|
||||
tsMap[sig] = newTs
|
||||
}
|
||||
|
||||
return sig
|
||||
}
|
||||
|
||||
// addExemplars finds a bucket bound that corresponds to the exemplars value and add the exemplar to the specific sig;
|
||||
// we only add exemplars if samples are presents
|
||||
// tsMap is unmodified if either of its parameters is nil and samples are nil.
|
||||
func addExemplars(tsMap map[string]*prompb.TimeSeries, exemplars []prompb.Exemplar, bucketBoundsData []bucketBoundsData) {
|
||||
if len(tsMap) == 0 || len(bucketBoundsData) == 0 || len(exemplars) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
sort.Sort(byBucketBoundsData(bucketBoundsData))
|
||||
|
||||
for _, exemplar := range exemplars {
|
||||
addExemplar(tsMap, bucketBoundsData, exemplar)
|
||||
}
|
||||
}
|
||||
|
||||
func addExemplar(tsMap map[string]*prompb.TimeSeries, bucketBounds []bucketBoundsData, exemplar prompb.Exemplar) {
|
||||
for _, bucketBound := range bucketBounds {
|
||||
sig := bucketBound.sig
|
||||
bound := bucketBound.bound
|
||||
|
||||
ts := tsMap[sig]
|
||||
if ts != nil && len(ts.Samples) > 0 && exemplar.Value <= bound {
|
||||
ts.Exemplars = append(ts.Exemplars, exemplar)
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// timeSeries return a string signature in the form of:
|
||||
//
|
||||
// TYPE-label1-value1- ... -labelN-valueN
|
||||
//
|
||||
// the label slice should not contain duplicate label names; this method sorts the slice by label name before creating
|
||||
// timeSeriesSignature returns a hashed label set signature.
|
||||
// The label slice should not contain duplicate label names; this method sorts the slice by label name before creating
|
||||
// the signature.
|
||||
func timeSeriesSignature(datatype string, labels []prompb.Label) string {
|
||||
length := len(datatype)
|
||||
|
||||
for _, lb := range labels {
|
||||
length += 2 + len(lb.GetName()) + len(lb.GetValue())
|
||||
}
|
||||
|
||||
b := strings.Builder{}
|
||||
b.Grow(length)
|
||||
b.WriteString(datatype)
|
||||
|
||||
// The algorithm is the same as in Prometheus' labels.StableHash function.
|
||||
func timeSeriesSignature(labels []prompb.Label) uint64 {
|
||||
sort.Sort(ByLabelName(labels))
|
||||
|
||||
for _, lb := range labels {
|
||||
b.WriteString("-")
|
||||
b.WriteString(lb.GetName())
|
||||
b.WriteString("-")
|
||||
b.WriteString(lb.GetValue())
|
||||
}
|
||||
// Use xxhash.Sum64(b) for fast path as it's faster.
|
||||
b := make([]byte, 0, 1024)
|
||||
for i, v := range labels {
|
||||
if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) {
|
||||
// If labels entry is 1KB+ do not allocate whole entry.
|
||||
h := xxhash.New()
|
||||
_, _ = h.Write(b)
|
||||
for _, v := range labels[i:] {
|
||||
_, _ = h.WriteString(v.Name)
|
||||
_, _ = h.Write(seps)
|
||||
_, _ = h.WriteString(v.Value)
|
||||
_, _ = h.Write(seps)
|
||||
}
|
||||
return h.Sum64()
|
||||
}
|
||||
|
||||
return b.String()
|
||||
b = append(b, v.Name...)
|
||||
b = append(b, seps[0])
|
||||
b = append(b, v.Value...)
|
||||
b = append(b, seps[0])
|
||||
}
|
||||
return xxhash.Sum64(b)
|
||||
}
|
||||
|
||||
var seps = []byte{'\xff'}
|
||||
|
||||
// createAttributes creates a slice of Prometheus Labels with OTLP attributes and pairs of string values.
|
||||
// Unpaired string values are ignored. String pairs overwrite OTLP labels if collisions happen, and overwrites are
|
||||
// logged. Resulting label names are sanitized.
|
||||
func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externalLabels map[string]string, extras ...string) []prompb.Label {
|
||||
serviceName, haveServiceName := resource.Attributes().Get(conventions.AttributeServiceName)
|
||||
instance, haveInstanceID := resource.Attributes().Get(conventions.AttributeServiceInstanceID)
|
||||
// Unpaired string values are ignored. String pairs overwrite OTLP labels if collisions happen and
|
||||
// if logOnOverwrite is true, the overwrite is logged. Resulting label names are sanitized.
|
||||
func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externalLabels map[string]string,
|
||||
ignoreAttrs []string, logOnOverwrite bool, extras ...string) []prompb.Label {
|
||||
resourceAttrs := resource.Attributes()
|
||||
serviceName, haveServiceName := resourceAttrs.Get(conventions.AttributeServiceName)
|
||||
instance, haveInstanceID := resourceAttrs.Get(conventions.AttributeServiceInstanceID)
|
||||
|
||||
// Calculate the maximum possible number of labels we could return so we can preallocate l
|
||||
maxLabelCount := attributes.Len() + len(externalLabels) + len(extras)/2
|
||||
|
@ -171,9 +137,13 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa
|
|||
|
||||
// Ensure attributes are sorted by key for consistent merging of keys which
|
||||
// collide when sanitized.
|
||||
labels := make([]prompb.Label, 0, attributes.Len())
|
||||
labels := make([]prompb.Label, 0, maxLabelCount)
|
||||
// XXX: Should we always drop service namespace/service name/service instance ID from the labels
|
||||
// (as they get mapped to other Prometheus labels)?
|
||||
attributes.Range(func(key string, value pcommon.Value) bool {
|
||||
labels = append(labels, prompb.Label{Name: key, Value: value.AsString()})
|
||||
if !slices.Contains(ignoreAttrs, key) {
|
||||
labels = append(labels, prompb.Label{Name: key, Value: value.AsString()})
|
||||
}
|
||||
return true
|
||||
})
|
||||
sort.Stable(ByLabelName(labels))
|
||||
|
@ -190,7 +160,7 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa
|
|||
// Map service.name + service.namespace to job
|
||||
if haveServiceName {
|
||||
val := serviceName.AsString()
|
||||
if serviceNamespace, ok := resource.Attributes().Get(conventions.AttributeServiceNamespace); ok {
|
||||
if serviceNamespace, ok := resourceAttrs.Get(conventions.AttributeServiceNamespace); ok {
|
||||
val = fmt.Sprintf("%s/%s", serviceNamespace.AsString(), val)
|
||||
}
|
||||
l[model.JobLabel] = val
|
||||
|
@ -213,7 +183,7 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa
|
|||
break
|
||||
}
|
||||
_, found := l[extras[i]]
|
||||
if found {
|
||||
if found && logOnOverwrite {
|
||||
log.Println("label " + extras[i] + " is overwritten. Check if Prometheus reserved labels are used.")
|
||||
}
|
||||
// internal labels should be maintained
|
||||
|
@ -224,12 +194,12 @@ func createAttributes(resource pcommon.Resource, attributes pcommon.Map, externa
|
|||
l[name] = extras[i+1]
|
||||
}
|
||||
|
||||
s := make([]prompb.Label, 0, len(l))
|
||||
labels = labels[:0]
|
||||
for k, v := range l {
|
||||
s = append(s, prompb.Label{Name: k, Value: v})
|
||||
labels = append(labels, prompb.Label{Name: k, Value: v})
|
||||
}
|
||||
|
||||
return s
|
||||
return labels
|
||||
}
|
||||
|
||||
// isValidAggregationTemporality checks whether an OTel metric has a valid
|
||||
|
@ -249,100 +219,84 @@ func isValidAggregationTemporality(metric pmetric.Metric) bool {
|
|||
return false
|
||||
}
|
||||
|
||||
// addSingleHistogramDataPoint converts pt to 2 + min(len(ExplicitBounds), len(BucketCount)) + 1 samples. It
|
||||
// ignore extra buckets if len(ExplicitBounds) > len(BucketCounts)
|
||||
func addSingleHistogramDataPoint(pt pmetric.HistogramDataPoint, resource pcommon.Resource, metric pmetric.Metric, settings Settings, tsMap map[string]*prompb.TimeSeries, baseName string) {
|
||||
timestamp := convertTimeStamp(pt.Timestamp())
|
||||
baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels)
|
||||
func (c *prometheusConverter) addHistogramDataPoints(dataPoints pmetric.HistogramDataPointSlice,
|
||||
resource pcommon.Resource, settings Settings, baseName string) {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
pt := dataPoints.At(x)
|
||||
timestamp := convertTimeStamp(pt.Timestamp())
|
||||
baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false)
|
||||
|
||||
createLabels := func(nameSuffix string, extras ...string) []prompb.Label {
|
||||
extraLabelCount := len(extras) / 2
|
||||
labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name
|
||||
copy(labels, baseLabels)
|
||||
// If the sum is unset, it indicates the _sum metric point should be
|
||||
// omitted
|
||||
if pt.HasSum() {
|
||||
// treat sum as a sample in an individual TimeSeries
|
||||
sum := &prompb.Sample{
|
||||
Value: pt.Sum(),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sum.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
|
||||
sumlabels := createLabels(baseName+sumStr, baseLabels)
|
||||
c.addSample(sum, sumlabels)
|
||||
|
||||
for extrasIdx := 0; extrasIdx < extraLabelCount; extrasIdx++ {
|
||||
labels = append(labels, prompb.Label{Name: extras[extrasIdx], Value: extras[extrasIdx+1]})
|
||||
}
|
||||
|
||||
// sum, count, and buckets of the histogram should append suffix to baseName
|
||||
labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: baseName + nameSuffix})
|
||||
|
||||
return labels
|
||||
}
|
||||
|
||||
// If the sum is unset, it indicates the _sum metric point should be
|
||||
// omitted
|
||||
if pt.HasSum() {
|
||||
// treat sum as a sample in an individual TimeSeries
|
||||
sum := &prompb.Sample{
|
||||
Value: pt.Sum(),
|
||||
// treat count as a sample in an individual TimeSeries
|
||||
count := &prompb.Sample{
|
||||
Value: float64(pt.Count()),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sum.Value = math.Float64frombits(value.StaleNaN)
|
||||
count.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
|
||||
sumlabels := createLabels(sumStr)
|
||||
addSample(tsMap, sum, sumlabels, metric.Type().String())
|
||||
countlabels := createLabels(baseName+countStr, baseLabels)
|
||||
c.addSample(count, countlabels)
|
||||
|
||||
}
|
||||
// cumulative count for conversion to cumulative histogram
|
||||
var cumulativeCount uint64
|
||||
|
||||
// treat count as a sample in an individual TimeSeries
|
||||
count := &prompb.Sample{
|
||||
Value: float64(pt.Count()),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
count.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
var bucketBounds []bucketBoundsData
|
||||
|
||||
countlabels := createLabels(countStr)
|
||||
addSample(tsMap, count, countlabels, metric.Type().String())
|
||||
// process each bound, based on histograms proto definition, # of buckets = # of explicit bounds + 1
|
||||
for i := 0; i < pt.ExplicitBounds().Len() && i < pt.BucketCounts().Len(); i++ {
|
||||
bound := pt.ExplicitBounds().At(i)
|
||||
cumulativeCount += pt.BucketCounts().At(i)
|
||||
bucket := &prompb.Sample{
|
||||
Value: float64(cumulativeCount),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
bucket.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
boundStr := strconv.FormatFloat(bound, 'f', -1, 64)
|
||||
labels := createLabels(baseName+bucketStr, baseLabels, leStr, boundStr)
|
||||
ts := c.addSample(bucket, labels)
|
||||
|
||||
// cumulative count for conversion to cumulative histogram
|
||||
var cumulativeCount uint64
|
||||
|
||||
promExemplars := getPromExemplars[pmetric.HistogramDataPoint](pt)
|
||||
|
||||
var bucketBounds []bucketBoundsData
|
||||
|
||||
// process each bound, based on histograms proto definition, # of buckets = # of explicit bounds + 1
|
||||
for i := 0; i < pt.ExplicitBounds().Len() && i < pt.BucketCounts().Len(); i++ {
|
||||
bound := pt.ExplicitBounds().At(i)
|
||||
cumulativeCount += pt.BucketCounts().At(i)
|
||||
bucket := &prompb.Sample{
|
||||
Value: float64(cumulativeCount),
|
||||
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: bound})
|
||||
}
|
||||
// add le=+Inf bucket
|
||||
infBucket := &prompb.Sample{
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
bucket.Value = math.Float64frombits(value.StaleNaN)
|
||||
infBucket.Value = math.Float64frombits(value.StaleNaN)
|
||||
} else {
|
||||
infBucket.Value = float64(pt.Count())
|
||||
}
|
||||
boundStr := strconv.FormatFloat(bound, 'f', -1, 64)
|
||||
labels := createLabels(bucketStr, leStr, boundStr)
|
||||
sig := addSample(tsMap, bucket, labels, metric.Type().String())
|
||||
infLabels := createLabels(baseName+bucketStr, baseLabels, leStr, pInfStr)
|
||||
ts := c.addSample(infBucket, infLabels)
|
||||
|
||||
bucketBounds = append(bucketBounds, bucketBoundsData{sig: sig, bound: bound})
|
||||
}
|
||||
// add le=+Inf bucket
|
||||
infBucket := &prompb.Sample{
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
infBucket.Value = math.Float64frombits(value.StaleNaN)
|
||||
} else {
|
||||
infBucket.Value = float64(pt.Count())
|
||||
}
|
||||
infLabels := createLabels(bucketStr, leStr, pInfStr)
|
||||
sig := addSample(tsMap, infBucket, infLabels, metric.Type().String())
|
||||
bucketBounds = append(bucketBounds, bucketBoundsData{ts: ts, bound: math.Inf(1)})
|
||||
c.addExemplars(pt, bucketBounds)
|
||||
|
||||
bucketBounds = append(bucketBounds, bucketBoundsData{sig: sig, bound: math.Inf(1)})
|
||||
addExemplars(tsMap, promExemplars, bucketBounds)
|
||||
|
||||
// add _created time series if needed
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if settings.ExportCreatedMetric && startTimestamp != 0 {
|
||||
labels := createLabels(createdSuffix)
|
||||
addCreatedTimeSeriesIfNeeded(tsMap, labels, startTimestamp, pt.Timestamp(), metric.Type().String())
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if settings.ExportCreatedMetric && startTimestamp != 0 {
|
||||
labels := createLabels(baseName+createdSuffix, baseLabels)
|
||||
c.addTimeSeriesIfNeeded(labels, startTimestamp, pt.Timestamp())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -448,129 +402,177 @@ func maxTimestamp(a, b pcommon.Timestamp) pcommon.Timestamp {
|
|||
return b
|
||||
}
|
||||
|
||||
// addSingleSummaryDataPoint converts pt to len(QuantileValues) + 2 samples.
|
||||
func addSingleSummaryDataPoint(pt pmetric.SummaryDataPoint, resource pcommon.Resource, metric pmetric.Metric, settings Settings,
|
||||
tsMap map[string]*prompb.TimeSeries, baseName string) {
|
||||
timestamp := convertTimeStamp(pt.Timestamp())
|
||||
baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels)
|
||||
func (c *prometheusConverter) addSummaryDataPoints(dataPoints pmetric.SummaryDataPointSlice, resource pcommon.Resource,
|
||||
settings Settings, baseName string) {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
pt := dataPoints.At(x)
|
||||
timestamp := convertTimeStamp(pt.Timestamp())
|
||||
baseLabels := createAttributes(resource, pt.Attributes(), settings.ExternalLabels, nil, false)
|
||||
|
||||
createLabels := func(name string, extras ...string) []prompb.Label {
|
||||
extraLabelCount := len(extras) / 2
|
||||
labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name
|
||||
copy(labels, baseLabels)
|
||||
|
||||
for extrasIdx := 0; extrasIdx < extraLabelCount; extrasIdx++ {
|
||||
labels = append(labels, prompb.Label{Name: extras[extrasIdx], Value: extras[extrasIdx+1]})
|
||||
}
|
||||
|
||||
labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: name})
|
||||
|
||||
return labels
|
||||
}
|
||||
|
||||
// treat sum as a sample in an individual TimeSeries
|
||||
sum := &prompb.Sample{
|
||||
Value: pt.Sum(),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sum.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
// sum and count of the summary should append suffix to baseName
|
||||
sumlabels := createLabels(baseName + sumStr)
|
||||
addSample(tsMap, sum, sumlabels, metric.Type().String())
|
||||
|
||||
// treat count as a sample in an individual TimeSeries
|
||||
count := &prompb.Sample{
|
||||
Value: float64(pt.Count()),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
count.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
countlabels := createLabels(baseName + countStr)
|
||||
addSample(tsMap, count, countlabels, metric.Type().String())
|
||||
|
||||
// process each percentile/quantile
|
||||
for i := 0; i < pt.QuantileValues().Len(); i++ {
|
||||
qt := pt.QuantileValues().At(i)
|
||||
quantile := &prompb.Sample{
|
||||
Value: qt.Value(),
|
||||
// treat sum as a sample in an individual TimeSeries
|
||||
sum := &prompb.Sample{
|
||||
Value: pt.Sum(),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
quantile.Value = math.Float64frombits(value.StaleNaN)
|
||||
sum.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64)
|
||||
qtlabels := createLabels(baseName, quantileStr, percentileStr)
|
||||
addSample(tsMap, quantile, qtlabels, metric.Type().String())
|
||||
}
|
||||
// sum and count of the summary should append suffix to baseName
|
||||
sumlabels := createLabels(baseName+sumStr, baseLabels)
|
||||
c.addSample(sum, sumlabels)
|
||||
|
||||
// add _created time series if needed
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if settings.ExportCreatedMetric && startTimestamp != 0 {
|
||||
createdLabels := createLabels(baseName + createdSuffix)
|
||||
addCreatedTimeSeriesIfNeeded(tsMap, createdLabels, startTimestamp, pt.Timestamp(), metric.Type().String())
|
||||
// treat count as a sample in an individual TimeSeries
|
||||
count := &prompb.Sample{
|
||||
Value: float64(pt.Count()),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
count.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
countlabels := createLabels(baseName+countStr, baseLabels)
|
||||
c.addSample(count, countlabels)
|
||||
|
||||
// process each percentile/quantile
|
||||
for i := 0; i < pt.QuantileValues().Len(); i++ {
|
||||
qt := pt.QuantileValues().At(i)
|
||||
quantile := &prompb.Sample{
|
||||
Value: qt.Value(),
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
quantile.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
percentileStr := strconv.FormatFloat(qt.Quantile(), 'f', -1, 64)
|
||||
qtlabels := createLabels(baseName, baseLabels, quantileStr, percentileStr)
|
||||
c.addSample(quantile, qtlabels)
|
||||
}
|
||||
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if settings.ExportCreatedMetric && startTimestamp != 0 {
|
||||
createdLabels := createLabels(baseName+createdSuffix, baseLabels)
|
||||
c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// addCreatedTimeSeriesIfNeeded adds {name}_created time series with a single
|
||||
// sample. If the series exists, then new samples won't be added.
|
||||
func addCreatedTimeSeriesIfNeeded(
|
||||
series map[string]*prompb.TimeSeries,
|
||||
labels []prompb.Label,
|
||||
startTimestamp pcommon.Timestamp,
|
||||
timestamp pcommon.Timestamp,
|
||||
metricType string,
|
||||
) {
|
||||
sig := timeSeriesSignature(metricType, labels)
|
||||
if _, ok := series[sig]; !ok {
|
||||
series[sig] = &prompb.TimeSeries{
|
||||
Labels: labels,
|
||||
Samples: []prompb.Sample{
|
||||
{ // convert ns to ms
|
||||
Value: float64(convertTimeStamp(startTimestamp)),
|
||||
Timestamp: convertTimeStamp(timestamp),
|
||||
},
|
||||
// createLabels returns a copy of baseLabels, adding to it the pair model.MetricNameLabel=name.
|
||||
// If extras are provided, corresponding label pairs are also added to the returned slice.
|
||||
// If extras is uneven length, the last (unpaired) extra will be ignored.
|
||||
func createLabels(name string, baseLabels []prompb.Label, extras ...string) []prompb.Label {
|
||||
extraLabelCount := len(extras) / 2
|
||||
labels := make([]prompb.Label, len(baseLabels), len(baseLabels)+extraLabelCount+1) // +1 for name
|
||||
copy(labels, baseLabels)
|
||||
|
||||
n := len(extras)
|
||||
n -= n % 2
|
||||
for extrasIdx := 0; extrasIdx < n; extrasIdx += 2 {
|
||||
labels = append(labels, prompb.Label{Name: extras[extrasIdx], Value: extras[extrasIdx+1]})
|
||||
}
|
||||
|
||||
labels = append(labels, prompb.Label{Name: model.MetricNameLabel, Value: name})
|
||||
return labels
|
||||
}
|
||||
|
||||
// getOrCreateTimeSeries returns the time series corresponding to the label set if existent, and false.
|
||||
// Otherwise it creates a new one and returns that, and true.
|
||||
func (c *prometheusConverter) getOrCreateTimeSeries(lbls []prompb.Label) (*prompb.TimeSeries, bool) {
|
||||
h := timeSeriesSignature(lbls)
|
||||
ts := c.unique[h]
|
||||
if ts != nil {
|
||||
if isSameMetric(ts, lbls) {
|
||||
// We already have this metric
|
||||
return ts, false
|
||||
}
|
||||
|
||||
// Look for a matching conflict
|
||||
for _, cTS := range c.conflicts[h] {
|
||||
if isSameMetric(cTS, lbls) {
|
||||
// We already have this metric
|
||||
return cTS, false
|
||||
}
|
||||
}
|
||||
|
||||
// New conflict
|
||||
ts = &prompb.TimeSeries{
|
||||
Labels: lbls,
|
||||
}
|
||||
c.conflicts[h] = append(c.conflicts[h], ts)
|
||||
return ts, true
|
||||
}
|
||||
|
||||
// This metric is new
|
||||
ts = &prompb.TimeSeries{
|
||||
Labels: lbls,
|
||||
}
|
||||
c.unique[h] = ts
|
||||
return ts, true
|
||||
}
|
||||
|
||||
// addTimeSeriesIfNeeded adds a corresponding time series if it doesn't already exist.
|
||||
// If the time series doesn't already exist, it gets added with startTimestamp for its value and timestamp for its timestamp,
|
||||
// both converted to milliseconds.
|
||||
func (c *prometheusConverter) addTimeSeriesIfNeeded(lbls []prompb.Label, startTimestamp pcommon.Timestamp, timestamp pcommon.Timestamp) {
|
||||
ts, created := c.getOrCreateTimeSeries(lbls)
|
||||
if created {
|
||||
ts.Samples = []prompb.Sample{
|
||||
{
|
||||
// convert ns to ms
|
||||
Value: float64(convertTimeStamp(startTimestamp)),
|
||||
Timestamp: convertTimeStamp(timestamp),
|
||||
},
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// addResourceTargetInfo converts the resource to the target info metric
|
||||
func addResourceTargetInfo(resource pcommon.Resource, settings Settings, timestamp pcommon.Timestamp, tsMap map[string]*prompb.TimeSeries) {
|
||||
if settings.DisableTargetInfo {
|
||||
// addResourceTargetInfo converts the resource to the target info metric.
|
||||
func addResourceTargetInfo(resource pcommon.Resource, settings Settings, timestamp pcommon.Timestamp, converter *prometheusConverter) {
|
||||
if settings.DisableTargetInfo || timestamp == 0 {
|
||||
return
|
||||
}
|
||||
// Use resource attributes (other than those used for job+instance) as the
|
||||
// metric labels for the target info metric
|
||||
attributes := pcommon.NewMap()
|
||||
resource.Attributes().CopyTo(attributes)
|
||||
attributes.RemoveIf(func(k string, _ pcommon.Value) bool {
|
||||
switch k {
|
||||
case conventions.AttributeServiceName, conventions.AttributeServiceNamespace, conventions.AttributeServiceInstanceID:
|
||||
// Remove resource attributes used for job + instance
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
|
||||
attributes := resource.Attributes()
|
||||
identifyingAttrs := []string{
|
||||
conventions.AttributeServiceNamespace,
|
||||
conventions.AttributeServiceName,
|
||||
conventions.AttributeServiceInstanceID,
|
||||
}
|
||||
nonIdentifyingAttrsCount := attributes.Len()
|
||||
for _, a := range identifyingAttrs {
|
||||
_, haveAttr := attributes.Get(a)
|
||||
if haveAttr {
|
||||
nonIdentifyingAttrsCount--
|
||||
}
|
||||
})
|
||||
if attributes.Len() == 0 {
|
||||
}
|
||||
if nonIdentifyingAttrsCount == 0 {
|
||||
// If we only have job + instance, then target_info isn't useful, so don't add it.
|
||||
return
|
||||
}
|
||||
// create parameters for addSample
|
||||
|
||||
name := targetMetricName
|
||||
if len(settings.Namespace) > 0 {
|
||||
name = settings.Namespace + "_" + name
|
||||
}
|
||||
labels := createAttributes(resource, attributes, settings.ExternalLabels, model.MetricNameLabel, name)
|
||||
|
||||
labels := createAttributes(resource, attributes, settings.ExternalLabels, identifyingAttrs, false, model.MetricNameLabel, name)
|
||||
haveIdentifier := false
|
||||
for _, l := range labels {
|
||||
if l.Name == model.JobLabel || l.Name == model.InstanceLabel {
|
||||
haveIdentifier = true
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if !haveIdentifier {
|
||||
// We need at least one identifying label to generate target_info.
|
||||
return
|
||||
}
|
||||
|
||||
sample := &prompb.Sample{
|
||||
Value: float64(1),
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(timestamp),
|
||||
}
|
||||
addSample(tsMap, sample, labels, infoType)
|
||||
converter.addSample(sample, labels)
|
||||
}
|
||||
|
||||
// convertTimeStamp converts OTLP timestamp in ns to timestamp in ms
|
||||
|
|
|
@ -1,58 +1,59 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/histograms.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheusremotewrite // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
|
||||
package prometheusremotewrite
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"go.opentelemetry.io/collector/pdata/pcommon"
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
)
|
||||
|
||||
const defaultZeroThreshold = 1e-128
|
||||
|
||||
func addSingleExponentialHistogramDataPoint(
|
||||
metric string,
|
||||
pt pmetric.ExponentialHistogramDataPoint,
|
||||
resource pcommon.Resource,
|
||||
settings Settings,
|
||||
series map[string]*prompb.TimeSeries,
|
||||
) error {
|
||||
labels := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
model.MetricNameLabel,
|
||||
metric,
|
||||
)
|
||||
func (c *prometheusConverter) addExponentialHistogramDataPoints(dataPoints pmetric.ExponentialHistogramDataPointSlice,
|
||||
resource pcommon.Resource, settings Settings, baseName string) error {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
pt := dataPoints.At(x)
|
||||
lbls := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
nil,
|
||||
true,
|
||||
model.MetricNameLabel,
|
||||
baseName,
|
||||
)
|
||||
ts, _ := c.getOrCreateTimeSeries(lbls)
|
||||
|
||||
sig := timeSeriesSignature(
|
||||
pmetric.MetricTypeExponentialHistogram.String(),
|
||||
labels,
|
||||
)
|
||||
ts, ok := series[sig]
|
||||
if !ok {
|
||||
ts = &prompb.TimeSeries{
|
||||
Labels: labels,
|
||||
histogram, err := exponentialToNativeHistogram(pt)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
series[sig] = ts
|
||||
}
|
||||
ts.Histograms = append(ts.Histograms, histogram)
|
||||
|
||||
histogram, err := exponentialToNativeHistogram(pt)
|
||||
if err != nil {
|
||||
return err
|
||||
exemplars := getPromExemplars[pmetric.ExponentialHistogramDataPoint](pt)
|
||||
ts.Exemplars = append(ts.Exemplars, exemplars...)
|
||||
}
|
||||
ts.Histograms = append(ts.Histograms, histogram)
|
||||
|
||||
exemplars := getPromExemplars[pmetric.ExponentialHistogramDataPoint](pt)
|
||||
ts.Exemplars = append(ts.Exemplars, exemplars...)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -1,19 +1,32 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/metrics_to_prw.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheusremotewrite // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
|
||||
package prometheusremotewrite
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strconv"
|
||||
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"go.opentelemetry.io/collector/pdata/pcommon"
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
"go.uber.org/multierr"
|
||||
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
prometheustranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
)
|
||||
|
||||
|
@ -27,9 +40,33 @@ type Settings struct {
|
|||
}
|
||||
|
||||
// FromMetrics converts pmetric.Metrics to Prometheus remote write format.
|
||||
func FromMetrics(md pmetric.Metrics, settings Settings) (tsMap map[string]*prompb.TimeSeries, errs error) {
|
||||
tsMap = make(map[string]*prompb.TimeSeries)
|
||||
func FromMetrics(md pmetric.Metrics, settings Settings) (map[string]*prompb.TimeSeries, error) {
|
||||
c := newPrometheusConverter()
|
||||
errs := c.fromMetrics(md, settings)
|
||||
tss := c.timeSeries()
|
||||
out := make(map[string]*prompb.TimeSeries, len(tss))
|
||||
for i := range tss {
|
||||
out[strconv.Itoa(i)] = &tss[i]
|
||||
}
|
||||
|
||||
return out, errs
|
||||
}
|
||||
|
||||
// prometheusConverter converts from OTel write format to Prometheus write format.
|
||||
type prometheusConverter struct {
|
||||
unique map[uint64]*prompb.TimeSeries
|
||||
conflicts map[uint64][]*prompb.TimeSeries
|
||||
}
|
||||
|
||||
func newPrometheusConverter() *prometheusConverter {
|
||||
return &prometheusConverter{
|
||||
unique: map[uint64]*prompb.TimeSeries{},
|
||||
conflicts: map[uint64][]*prompb.TimeSeries{},
|
||||
}
|
||||
}
|
||||
|
||||
// fromMetrics converts pmetric.Metrics to Prometheus remote write format.
|
||||
func (c *prometheusConverter) fromMetrics(md pmetric.Metrics, settings Settings) (errs error) {
|
||||
resourceMetricsSlice := md.ResourceMetrics()
|
||||
for i := 0; i < resourceMetricsSlice.Len(); i++ {
|
||||
resourceMetrics := resourceMetricsSlice.At(i)
|
||||
|
@ -39,8 +76,7 @@ func FromMetrics(md pmetric.Metrics, settings Settings) (tsMap map[string]*promp
|
|||
// use with the "target" info metric
|
||||
var mostRecentTimestamp pcommon.Timestamp
|
||||
for j := 0; j < scopeMetricsSlice.Len(); j++ {
|
||||
scopeMetrics := scopeMetricsSlice.At(j)
|
||||
metricSlice := scopeMetrics.Metrics()
|
||||
metricSlice := scopeMetricsSlice.At(j).Metrics()
|
||||
|
||||
// TODO: decide if instrumentation library information should be exported as labels
|
||||
for k := 0; k < metricSlice.Len(); k++ {
|
||||
|
@ -54,65 +90,125 @@ func FromMetrics(md pmetric.Metrics, settings Settings) (tsMap map[string]*promp
|
|||
|
||||
promName := prometheustranslator.BuildCompliantName(metric, settings.Namespace, settings.AddMetricSuffixes)
|
||||
|
||||
// handle individual metric based on type
|
||||
// handle individual metrics based on type
|
||||
//exhaustive:enforce
|
||||
switch metric.Type() {
|
||||
case pmetric.MetricTypeGauge:
|
||||
dataPoints := metric.Gauge().DataPoints()
|
||||
if dataPoints.Len() == 0 {
|
||||
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
|
||||
break
|
||||
}
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
addSingleGaugeNumberDataPoint(dataPoints.At(x), resource, metric, settings, tsMap, promName)
|
||||
}
|
||||
c.addGaugeNumberDataPoints(dataPoints, resource, settings, promName)
|
||||
case pmetric.MetricTypeSum:
|
||||
dataPoints := metric.Sum().DataPoints()
|
||||
if dataPoints.Len() == 0 {
|
||||
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
|
||||
break
|
||||
}
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
addSingleSumNumberDataPoint(dataPoints.At(x), resource, metric, settings, tsMap, promName)
|
||||
}
|
||||
c.addSumNumberDataPoints(dataPoints, resource, metric, settings, promName)
|
||||
case pmetric.MetricTypeHistogram:
|
||||
dataPoints := metric.Histogram().DataPoints()
|
||||
if dataPoints.Len() == 0 {
|
||||
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
|
||||
break
|
||||
}
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
addSingleHistogramDataPoint(dataPoints.At(x), resource, metric, settings, tsMap, promName)
|
||||
}
|
||||
c.addHistogramDataPoints(dataPoints, resource, settings, promName)
|
||||
case pmetric.MetricTypeExponentialHistogram:
|
||||
dataPoints := metric.ExponentialHistogram().DataPoints()
|
||||
if dataPoints.Len() == 0 {
|
||||
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
|
||||
break
|
||||
}
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
errs = multierr.Append(
|
||||
errs,
|
||||
addSingleExponentialHistogramDataPoint(
|
||||
promName,
|
||||
dataPoints.At(x),
|
||||
resource,
|
||||
settings,
|
||||
tsMap,
|
||||
),
|
||||
)
|
||||
}
|
||||
errs = multierr.Append(errs, c.addExponentialHistogramDataPoints(
|
||||
dataPoints,
|
||||
resource,
|
||||
settings,
|
||||
promName,
|
||||
))
|
||||
case pmetric.MetricTypeSummary:
|
||||
dataPoints := metric.Summary().DataPoints()
|
||||
if dataPoints.Len() == 0 {
|
||||
errs = multierr.Append(errs, fmt.Errorf("empty data points. %s is dropped", metric.Name()))
|
||||
break
|
||||
}
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
addSingleSummaryDataPoint(dataPoints.At(x), resource, metric, settings, tsMap, promName)
|
||||
}
|
||||
c.addSummaryDataPoints(dataPoints, resource, settings, promName)
|
||||
default:
|
||||
errs = multierr.Append(errs, errors.New("unsupported metric type"))
|
||||
}
|
||||
}
|
||||
}
|
||||
addResourceTargetInfo(resource, settings, mostRecentTimestamp, tsMap)
|
||||
addResourceTargetInfo(resource, settings, mostRecentTimestamp, c)
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
// timeSeries returns a slice of the prompb.TimeSeries that were converted from OTel format.
|
||||
func (c *prometheusConverter) timeSeries() []prompb.TimeSeries {
|
||||
conflicts := 0
|
||||
for _, ts := range c.conflicts {
|
||||
conflicts += len(ts)
|
||||
}
|
||||
allTS := make([]prompb.TimeSeries, 0, len(c.unique)+conflicts)
|
||||
for _, ts := range c.unique {
|
||||
allTS = append(allTS, *ts)
|
||||
}
|
||||
for _, cTS := range c.conflicts {
|
||||
for _, ts := range cTS {
|
||||
allTS = append(allTS, *ts)
|
||||
}
|
||||
}
|
||||
|
||||
return allTS
|
||||
}
|
||||
|
||||
func isSameMetric(ts *prompb.TimeSeries, lbls []prompb.Label) bool {
|
||||
if len(ts.Labels) != len(lbls) {
|
||||
return false
|
||||
}
|
||||
for i, l := range ts.Labels {
|
||||
if l.Name != ts.Labels[i].Name || l.Value != ts.Labels[i].Value {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// addExemplars adds exemplars for the dataPoint. For each exemplar, if it can find a bucket bound corresponding to its value,
|
||||
// the exemplar is added to the bucket bound's time series, provided that the time series' has samples.
|
||||
func (c *prometheusConverter) addExemplars(dataPoint pmetric.HistogramDataPoint, bucketBounds []bucketBoundsData) {
|
||||
if len(bucketBounds) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
exemplars := getPromExemplars(dataPoint)
|
||||
if len(exemplars) == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
sort.Sort(byBucketBoundsData(bucketBounds))
|
||||
for _, exemplar := range exemplars {
|
||||
for _, bound := range bucketBounds {
|
||||
if len(bound.ts.Samples) > 0 && exemplar.Value <= bound.bound {
|
||||
bound.ts.Exemplars = append(bound.ts.Exemplars, exemplar)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// addSample finds a TimeSeries that corresponds to lbls, and adds sample to it.
|
||||
// If there is no corresponding TimeSeries already, it's created.
|
||||
// The corresponding TimeSeries is returned.
|
||||
// If either lbls is nil/empty or sample is nil, nothing is done.
|
||||
func (c *prometheusConverter) addSample(sample *prompb.Sample, lbls []prompb.Label) *prompb.TimeSeries {
|
||||
if sample == nil || len(lbls) == 0 {
|
||||
// This shouldn't happen
|
||||
return nil
|
||||
}
|
||||
|
||||
ts, _ := c.getOrCreateTimeSeries(lbls)
|
||||
ts.Samples = append(ts.Samples, *sample)
|
||||
return ts
|
||||
}
|
||||
|
|
|
@ -1,106 +1,110 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/number_data_points.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheusremotewrite // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
|
||||
package prometheusremotewrite
|
||||
|
||||
import (
|
||||
"math"
|
||||
|
||||
"github.com/prometheus/common/model"
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"go.opentelemetry.io/collector/pdata/pcommon"
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"github.com/prometheus/prometheus/model/value"
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
)
|
||||
|
||||
// addSingleGaugeNumberDataPoint converts the Gauge metric data point to a
|
||||
// Prometheus time series with samples and labels. The result is stored in the
|
||||
// series map.
|
||||
func addSingleGaugeNumberDataPoint(
|
||||
pt pmetric.NumberDataPoint,
|
||||
resource pcommon.Resource,
|
||||
metric pmetric.Metric,
|
||||
settings Settings,
|
||||
series map[string]*prompb.TimeSeries,
|
||||
name string,
|
||||
) {
|
||||
labels := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
model.MetricNameLabel,
|
||||
name,
|
||||
)
|
||||
sample := &prompb.Sample{
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(pt.Timestamp()),
|
||||
func (c *prometheusConverter) addGaugeNumberDataPoints(dataPoints pmetric.NumberDataPointSlice,
|
||||
resource pcommon.Resource, settings Settings, name string) {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
pt := dataPoints.At(x)
|
||||
labels := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
nil,
|
||||
true,
|
||||
model.MetricNameLabel,
|
||||
name,
|
||||
)
|
||||
sample := &prompb.Sample{
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(pt.Timestamp()),
|
||||
}
|
||||
switch pt.ValueType() {
|
||||
case pmetric.NumberDataPointValueTypeInt:
|
||||
sample.Value = float64(pt.IntValue())
|
||||
case pmetric.NumberDataPointValueTypeDouble:
|
||||
sample.Value = pt.DoubleValue()
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sample.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
c.addSample(sample, labels)
|
||||
}
|
||||
switch pt.ValueType() {
|
||||
case pmetric.NumberDataPointValueTypeInt:
|
||||
sample.Value = float64(pt.IntValue())
|
||||
case pmetric.NumberDataPointValueTypeDouble:
|
||||
sample.Value = pt.DoubleValue()
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sample.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
addSample(series, sample, labels, metric.Type().String())
|
||||
}
|
||||
|
||||
// addSingleSumNumberDataPoint converts the Sum metric data point to a Prometheus
|
||||
// time series with samples, labels and exemplars. The result is stored in the
|
||||
// series map.
|
||||
func addSingleSumNumberDataPoint(
|
||||
pt pmetric.NumberDataPoint,
|
||||
resource pcommon.Resource,
|
||||
metric pmetric.Metric,
|
||||
settings Settings,
|
||||
series map[string]*prompb.TimeSeries,
|
||||
name string,
|
||||
) {
|
||||
labels := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
model.MetricNameLabel, name,
|
||||
)
|
||||
sample := &prompb.Sample{
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(pt.Timestamp()),
|
||||
}
|
||||
switch pt.ValueType() {
|
||||
case pmetric.NumberDataPointValueTypeInt:
|
||||
sample.Value = float64(pt.IntValue())
|
||||
case pmetric.NumberDataPointValueTypeDouble:
|
||||
sample.Value = pt.DoubleValue()
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sample.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
sig := addSample(series, sample, labels, metric.Type().String())
|
||||
|
||||
if ts := series[sig]; sig != "" && ts != nil {
|
||||
exemplars := getPromExemplars[pmetric.NumberDataPoint](pt)
|
||||
ts.Exemplars = append(ts.Exemplars, exemplars...)
|
||||
}
|
||||
|
||||
// add _created time series if needed
|
||||
if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() {
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if startTimestamp == 0 {
|
||||
return
|
||||
func (c *prometheusConverter) addSumNumberDataPoints(dataPoints pmetric.NumberDataPointSlice,
|
||||
resource pcommon.Resource, metric pmetric.Metric, settings Settings, name string) {
|
||||
for x := 0; x < dataPoints.Len(); x++ {
|
||||
pt := dataPoints.At(x)
|
||||
lbls := createAttributes(
|
||||
resource,
|
||||
pt.Attributes(),
|
||||
settings.ExternalLabels,
|
||||
nil,
|
||||
true,
|
||||
model.MetricNameLabel,
|
||||
name,
|
||||
)
|
||||
sample := &prompb.Sample{
|
||||
// convert ns to ms
|
||||
Timestamp: convertTimeStamp(pt.Timestamp()),
|
||||
}
|
||||
switch pt.ValueType() {
|
||||
case pmetric.NumberDataPointValueTypeInt:
|
||||
sample.Value = float64(pt.IntValue())
|
||||
case pmetric.NumberDataPointValueTypeDouble:
|
||||
sample.Value = pt.DoubleValue()
|
||||
}
|
||||
if pt.Flags().NoRecordedValue() {
|
||||
sample.Value = math.Float64frombits(value.StaleNaN)
|
||||
}
|
||||
ts := c.addSample(sample, lbls)
|
||||
if ts != nil {
|
||||
exemplars := getPromExemplars[pmetric.NumberDataPoint](pt)
|
||||
ts.Exemplars = append(ts.Exemplars, exemplars...)
|
||||
}
|
||||
|
||||
createdLabels := make([]prompb.Label, len(labels))
|
||||
copy(createdLabels, labels)
|
||||
for i, l := range createdLabels {
|
||||
if l.Name == model.MetricNameLabel {
|
||||
createdLabels[i].Value = name + createdSuffix
|
||||
break
|
||||
// add created time series if needed
|
||||
if settings.ExportCreatedMetric && metric.Sum().IsMonotonic() {
|
||||
startTimestamp := pt.StartTimestamp()
|
||||
if startTimestamp == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
createdLabels := make([]prompb.Label, len(lbls))
|
||||
copy(createdLabels, lbls)
|
||||
for i, l := range createdLabels {
|
||||
if l.Name == model.MetricNameLabel {
|
||||
createdLabels[i].Value = name + createdSuffix
|
||||
break
|
||||
}
|
||||
}
|
||||
c.addTimeSeriesIfNeeded(createdLabels, startTimestamp, pt.Timestamp())
|
||||
}
|
||||
addCreatedTimeSeriesIfNeeded(series, createdLabels, startTimestamp, pt.Timestamp(), metric.Type().String())
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,14 +1,25 @@
|
|||
// DO NOT EDIT. COPIED AS-IS. SEE ../README.md
|
||||
// Copyright 2024 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.
|
||||
// Provenance-includes-location: https://github.com/open-telemetry/opentelemetry-collector-contrib/blob/95e8f8fdc2a9dc87230406c9a3cf02be4fd68bea/pkg/translator/prometheusremotewrite/otlp_to_openmetrics_metadata.go
|
||||
// Provenance-includes-license: Apache-2.0
|
||||
// Provenance-includes-copyright: Copyright The OpenTelemetry Authors.
|
||||
|
||||
// Copyright The OpenTelemetry Authors
|
||||
// SPDX-License-Identifier: Apache-2.0
|
||||
|
||||
package prometheusremotewrite // import "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheusremotewrite"
|
||||
package prometheusremotewrite
|
||||
|
||||
import (
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
"go.opentelemetry.io/collector/pdata/pmetric"
|
||||
|
||||
"github.com/prometheus/prometheus/prompb"
|
||||
prometheustranslator "github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus"
|
||||
)
|
||||
|
||||
|
|
|
@ -1,27 +0,0 @@
|
|||
#!/bin/bash
|
||||
set -xe
|
||||
|
||||
OTEL_VERSION=v0.95.0
|
||||
|
||||
git clone https://github.com/open-telemetry/opentelemetry-collector-contrib ./tmp
|
||||
cd ./tmp
|
||||
git checkout $OTEL_VERSION
|
||||
cd ..
|
||||
|
||||
rm -rf ./prometheusremotewrite/*
|
||||
cp -r ./tmp/pkg/translator/prometheusremotewrite/*.go ./prometheusremotewrite
|
||||
rm -rf ./prometheusremotewrite/*_test.go
|
||||
|
||||
rm -rf ./prometheus/*
|
||||
cp -r ./tmp/pkg/translator/prometheus/*.go ./prometheus
|
||||
rm -rf ./prometheus/*_test.go
|
||||
|
||||
rm -rf ./tmp
|
||||
|
||||
case $(sed --help 2>&1) in
|
||||
*GNU*) set sed -i;;
|
||||
*) set sed -i '';;
|
||||
esac
|
||||
|
||||
"$@" -e 's#github.com/open-telemetry/opentelemetry-collector-contrib/pkg/translator/prometheus#github.com/prometheus/prometheus/storage/remote/otlptranslator/prometheus#g' ./prometheusremotewrite/*.go ./prometheus/*.go
|
||||
"$@" -e '1s#^#// DO NOT EDIT. COPIED AS-IS. SEE ../README.md\n\n#g' ./prometheusremotewrite/*.go ./prometheus/*.go
|
Loading…
Reference in New Issue