Skip to content

Commit

Permalink
Add trace head-sampling (#583) (#23)
Browse files Browse the repository at this point in the history
Cherry-picking census-instrumentation/opencensus-service#583 from OC.

* Add trace head-sampling

This is to complete the sampling feature since OC Service already offers tail-sampling. This is implemented using an extra hashing as an attempt to avoid bias from trace ID generation and also to correctly spread traces for backends that also do hashing.

Tests to enforce hash of different lengths.

Make hashing seed configurable

* Direct recommendation about hash seeding setting

* More precise wording for hash-seed comment
Paulo Janotti authored Jun 19, 2019

Verified

This commit was created on GitHub.com and signed with GitHub’s verified signature. The key has expired.
1 parent d41326d commit 2a40f54
Showing 6 changed files with 564 additions and 5 deletions.
32 changes: 28 additions & 4 deletions README.md
Original file line number Diff line number Diff line change
@@ -20,7 +20,7 @@ For now, please use the [OpenCensus Service](https://github.com/open-telemetry/o
- [Exporters](#config-exporters)
- [Diagnostics](#config-diagnostics)
- [Global Attributes](#global-attributes)
- [Intelligent Sampling](#tail-sampling)
- [Sampling](#sampling)
- [Usage](#usage)

## Introduction
@@ -243,9 +243,33 @@ global:

### <a name="sampling"></a>Sampling

Sampling can also be configured on the OpenTelemetry Service. Tail sampling
must be configured on the Collector as it requires all spans for a given trace
to make a sampling decision.
Sampling can also be configured on the OpenTelemetry Service. Both head-based and
tail-based sampling are supported. Either the Agent or the Collector may enable
head-based sampling. Tail sampling must be configured on the Collector as it
requires all spans for a given trace to make a sampling decision.

#### Head-based Example

```yaml
sampling:
# mode indicates if the sampling is head or tail based. For probabilistic the mode is head-based.
mode: head
policies:
# section below defines a probabilistic trace sampler based on hashing the trace ID associated to
# each span and sampling the span according to the given spans.
probabilistic:
configuration:
# sampling-percentage is the percentage of sampling to be applied to all spans, unless their service is specified
# on sampling-percentage.
sampling-percentage: 5
# hash-seed allows choosing the seed for the hash function used in the trace sampling. This is important when
# multiple layers of collectors are being used with head sampling, in such scenarios make sure to
# choose different seeds for each layer.
hash-seed: 1
```

#### Tail-based Example

```yaml
sampling:
mode: tail
3 changes: 3 additions & 0 deletions cmd/occollector/app/builder/sampling_builder.go
Original file line number Diff line number Diff line change
@@ -33,6 +33,9 @@ const (
// NoSampling mode is the default and means that all data arriving at the collector
// is passed ahead.
NoSampling Mode = "no-sampling"
// HeadSampling is the mode in which trace data is sampled at ingestion, without seeing
// the whole trace data.
HeadSampling Mode = "head"
// TailSampling is the mode in which trace data is temporarily retained until an evaluation
// if the trace should be sampled is performed.
TailSampling Mode = "tail"
29 changes: 28 additions & 1 deletion cmd/occollector/app/collector/processors.go
Original file line number Diff line number Diff line change
@@ -35,6 +35,7 @@ import (
"github.com/open-telemetry/opentelemetry-service/processor/addattributesprocessor"
"github.com/open-telemetry/opentelemetry-service/processor/attributekeyprocessor"
"github.com/open-telemetry/opentelemetry-service/processor/multiconsumer"
"github.com/open-telemetry/opentelemetry-service/processor/tracesamplerprocessor"
)

func createExporters(v *viper.Viper, logger *zap.Logger) ([]func(), []consumer.TraceConsumer, []consumer.MetricsConsumer) {
@@ -281,7 +282,12 @@ func startProcessor(v *viper.Viper, logger *zap.Logger) (consumer.TraceConsumer,

var tailSamplingProcessor consumer.TraceConsumer
samplingProcessorCfg := builder.NewDefaultSamplingCfg().InitFromViper(v)
if samplingProcessorCfg.Mode == builder.TailSampling {
useHeadSamplingProcessor := false
if samplingProcessorCfg.Mode == builder.HeadSampling {
// Head-sampling should be the first processor in the pipeline to avoid global operations on data
// that is not going to be sampled, for now just set a flag to added the sampler later.
useHeadSamplingProcessor = true
} else if samplingProcessorCfg.Mode == builder.TailSampling {
var err error
tailSamplingProcessor, err = buildSamplingProcessor(samplingProcessorCfg, nameToTraceConsumer, v, logger)
if err != nil {
@@ -331,5 +337,26 @@ func startProcessor(v *viper.Viper, logger *zap.Logger) (consumer.TraceConsumer,
tp, _ = attributekeyprocessor.NewTraceProcessor(tp, multiProcessorCfg.Global.Attributes.KeyReplacements...)
}
}

if useHeadSamplingProcessor {
vTraceSampler := v.Sub("sampling.policies.probabilistic.configuration")
if vTraceSampler == nil {
logger.Error("Trace head-based sampling mode is enabled but there is no valid policy section defined")
os.Exit(1)
}

cfg := &tracesamplerprocessor.TraceSamplerCfg{}
samplerCfg, err := cfg.InitFromViper(vTraceSampler)
if err != nil {
logger.Error("Trace head-based sampling configuration error", zap.Error(err))
os.Exit(1)
}
logger.Info(
"Trace head-sampling enabled",
zap.Float32("sampling-percentage", samplerCfg.SamplingPercentage),
)
tp, _ = tracesamplerprocessor.NewTraceProcessor(tp, *samplerCfg)
}

return tp, closeFns
}
19 changes: 19 additions & 0 deletions cmd/occollector/app/collector/processors_test.go
Original file line number Diff line number Diff line change
@@ -25,6 +25,7 @@ import (
"github.com/open-telemetry/opentelemetry-service/processor/attributekeyprocessor"
"github.com/open-telemetry/opentelemetry-service/processor/multiconsumer"
"github.com/open-telemetry/opentelemetry-service/processor/processortest"
"github.com/open-telemetry/opentelemetry-service/processor/tracesamplerprocessor"
)

func Test_startProcessor(t *testing.T) {
@@ -85,6 +86,24 @@ func Test_startProcessor(t *testing.T) {
return attributeKeyProcessor
},
},
{
name: "sampling_config_trace_sampler",
setupViperCfg: func() *viper.Viper {
v := viper.New()
v.Set("logging-exporter", true)
v.Set("sampling.mode", "head")
v.Set("sampling.policies.probabilistic.configuration.sampling-percentage", 5)
return v
},
wantExamplar: func(t *testing.T) interface{} {
nopProcessor := processortest.NewNopTraceProcessor(nil)
tracesamplerprocessor, err := tracesamplerprocessor.NewTraceProcessor(nopProcessor, tracesamplerprocessor.TraceSamplerCfg{})
if err != nil {
t.Fatalf("tracesamplerprocessor.NewTraceProcessor() = %v", err)
}
return tracesamplerprocessor
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
168 changes: 168 additions & 0 deletions processor/tracesamplerprocessor/tracesamplerprocessor.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,168 @@
// Copyright 2019, OpenCensus 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 tracesamplerprocessor

import (
"context"
"errors"
"fmt"

tracepb "github.com/census-instrumentation/opencensus-proto/gen-go/trace/v1"
"github.com/spf13/viper"

"github.com/open-telemetry/opentelemetry-service/consumer"
"github.com/open-telemetry/opentelemetry-service/data"
"github.com/open-telemetry/opentelemetry-service/processor"
)

const (
// The constants below are tags used to read the configuration via viper.
samplingPercentageCfgTag = "sampling-percentage"
hashSeedCfgTag = "hash-seed"

// The constants help translate user friendly percentages to numbers direct used in sampling.
numHashBuckets = 0x4000 // Using a power of 2 to avoid division.
bitMaskHashBuckets = numHashBuckets - 1
percentageScaleFactor = numHashBuckets / 100.0
)

// TraceSamplerCfg has the configuration guiding the trace sampler processor.
type TraceSamplerCfg struct {
// SamplingPercentage is the percentage rate at which traces are going to be sampled. Defaults to zero, i.e.: no sample.
// Values greater or equal 100 are treated as "sample all traces".
SamplingPercentage float32
// HashSeed allows one to configure the hashing seed. This is important in scenarios where multiple layers of collectors
// have different sampling rates: if they use the same seed all passing one layer may pass the other even if they have
// different sampling rates, configuring different seeds avoids that.
HashSeed uint32
}

// InitFromViper updates TraceSamplerCfg according to the viper configuration.
func (tsc *TraceSamplerCfg) InitFromViper(v *viper.Viper) (*TraceSamplerCfg, error) {
if v == nil {
return nil, errors.New("v is nil")
}
if err := v.UnmarshalKey(samplingPercentageCfgTag, &tsc.SamplingPercentage); err != nil {
return nil, fmt.Errorf("failed to unmarshal %q: %v", samplingPercentageCfgTag, err)
}
if err := v.UnmarshalKey(hashSeedCfgTag, &tsc.HashSeed); err != nil {
return nil, fmt.Errorf("failed to unmarshal %q: %v", hashSeedCfgTag, err)
}
return tsc, nil
}

type tracesamplerprocessor struct {
nextConsumer consumer.TraceConsumer
scaledSamplingRate uint32
hashSeed uint32
}

var _ processor.TraceProcessor = (*tracesamplerprocessor)(nil)

// NewTraceProcessor returns a processor.TraceProcessor that will perform head sampling according to the given
// configuration.
func NewTraceProcessor(nextConsumer consumer.TraceConsumer, cfg TraceSamplerCfg) (processor.TraceProcessor, error) {
if nextConsumer == nil {
return nil, errors.New("nextConsumer is nil")
}

return &tracesamplerprocessor{
nextConsumer: nextConsumer,
// Adjust sampling percentage on private so recalculations are avoided.
scaledSamplingRate: uint32(cfg.SamplingPercentage * percentageScaleFactor),
hashSeed: cfg.HashSeed,
}, nil
}

func (tsp *tracesamplerprocessor) ConsumeTraceData(ctx context.Context, td data.TraceData) error {
scaledSamplingRate := tsp.scaledSamplingRate
if scaledSamplingRate >= numHashBuckets {
return tsp.nextConsumer.ConsumeTraceData(ctx, td)
}

sampledTraceData := data.TraceData{
Node: td.Node,
Resource: td.Resource,
SourceFormat: td.SourceFormat,
}

sampledSpans := make([]*tracepb.Span, 0, len(td.Spans))
for _, span := range td.Spans {
// If one assumes random trace ids hashing may seems avoidable, however, traces can be coming from sources
// with various different criterias to generate trace id and perhaps were already sampled without hashing.
// Hashing here prevents bias due to such systems.
if hash(span.TraceId, tsp.hashSeed)&bitMaskHashBuckets < scaledSamplingRate {
sampledSpans = append(sampledSpans, span)
}
}

sampledTraceData.Spans = sampledSpans

return tsp.nextConsumer.ConsumeTraceData(ctx, sampledTraceData)
}

// hash is a murmur3 hash function, see http://en.wikipedia.org/wiki/MurmurHash.
func hash(key []byte, seed uint32) (hash uint32) {
const (
c1 = 0xcc9e2d51
c2 = 0x1b873593
c3 = 0x85ebca6b
c4 = 0xc2b2ae35
r1 = 15
r2 = 13
m = 5
n = 0xe6546b64
)

hash = seed
iByte := 0
for ; iByte+4 <= len(key); iByte += 4 {
k := uint32(key[iByte]) | uint32(key[iByte+1])<<8 | uint32(key[iByte+2])<<16 | uint32(key[iByte+3])<<24
k *= c1
k = (k << r1) | (k >> (32 - r1))
k *= c2
hash ^= k
hash = (hash << r2) | (hash >> (32 - r2))
hash = hash*m + n
}

// TraceId and SpanId have lengths that are multiple of 4 so the code below is never expected to
// be hit when sampling traces. However, it is preserved here to keep it as a correct murmur3 implementation.
// This is enforced via tests.
var remainingBytes uint32
switch len(key) - iByte {
case 3:
remainingBytes += uint32(key[iByte+2]) << 16
fallthrough
case 2:
remainingBytes += uint32(key[iByte+1]) << 8
fallthrough
case 1:
remainingBytes += uint32(key[iByte])
remainingBytes *= c1
remainingBytes = (remainingBytes << r1) | (remainingBytes >> (32 - r1))
remainingBytes = remainingBytes * c2
hash ^= remainingBytes
}

hash ^= uint32(len(key))
hash ^= hash >> 16
hash *= c3
hash ^= hash >> 13
hash *= c4
hash ^= hash >> 16

return
}
318 changes: 318 additions & 0 deletions processor/tracesamplerprocessor/tracesamplerprocessor_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,318 @@
// Copyright 2019, OpenCensus 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 tracesamplerprocessor

import (
"context"
"math"
"math/rand"
"reflect"
"testing"

"github.com/spf13/viper"

commonpb "github.com/census-instrumentation/opencensus-proto/gen-go/agent/common/v1"
tracepb "github.com/census-instrumentation/opencensus-proto/gen-go/trace/v1"
"github.com/open-telemetry/opentelemetry-service/consumer"
"github.com/open-telemetry/opentelemetry-service/data"
"github.com/open-telemetry/opentelemetry-service/exporter/exportertest"
processormetrics "github.com/open-telemetry/opentelemetry-service/internal/collector/processor"
"github.com/open-telemetry/opentelemetry-service/processor"
tracetranslator "github.com/open-telemetry/opentelemetry-service/translator/trace"
)

func TestTraceSamplerCfg_InitFromViper(t *testing.T) {
type fields struct {
SamplingPercentage float32
}
tests := []struct {
name string
fields fields
genViperFn func() *viper.Viper
want *TraceSamplerCfg
wantErr bool
}{
{
name: "nil_viper",
wantErr: true,
genViperFn: func() *viper.Viper {
return nil
},
},
{
name: "invalid_sampling_rate",
wantErr: true,
genViperFn: func() *viper.Viper {
v := viper.New()
v.Set(samplingPercentageCfgTag, "value.is.not.a.number")
return v
},
},
{
name: "invalid_hash_seed",
wantErr: true,
genViperFn: func() *viper.Viper {
v := viper.New()
v.Set(hashSeedCfgTag, "value.is.not.a.number")
return v
},
},
{
name: "happy_path",
genViperFn: func() *viper.Viper {
v := viper.New()
v.Set(samplingPercentageCfgTag, 5)
return v
},
want: &TraceSamplerCfg{
SamplingPercentage: 5.0,
},
},
{
name: "happy_path_custom_seed",
genViperFn: func() *viper.Viper {
v := viper.New()
v.Set(samplingPercentageCfgTag, 0.03)
v.Set(hashSeedCfgTag, 1234)
return v
},
want: &TraceSamplerCfg{
SamplingPercentage: 0.03,
HashSeed: 1234,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
tsc := &TraceSamplerCfg{
SamplingPercentage: tt.fields.SamplingPercentage,
}
got, err := tsc.InitFromViper(tt.genViperFn())
if (err != nil) != tt.wantErr {
t.Errorf("TraceSamplerCfg.InitFromViper() error = %v, wantErr %v", err, tt.wantErr)
return
}
if !reflect.DeepEqual(got, tt.want) {
t.Errorf("TraceSamplerCfg.InitFromViper() = %v, want %v", got, tt.want)
}
})
}
}

func TestNewTraceProcessor(t *testing.T) {
tests := []struct {
name string
nextConsumer consumer.TraceConsumer
cfg TraceSamplerCfg
want processor.TraceProcessor
wantErr bool
}{
{
name: "nil_nextConsumer",
wantErr: true,
},
{
name: "happy_path",
nextConsumer: &exportertest.SinkTraceExporter{},
cfg: TraceSamplerCfg{
SamplingPercentage: 15.5,
},
want: &tracesamplerprocessor{
nextConsumer: &exportertest.SinkTraceExporter{},
},
},
{
name: "happy_path_hash_seed",
nextConsumer: &exportertest.SinkTraceExporter{},
cfg: TraceSamplerCfg{
SamplingPercentage: 13.33,
HashSeed: 4321,
},
want: &tracesamplerprocessor{
nextConsumer: &exportertest.SinkTraceExporter{},
hashSeed: 4321,
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
if !tt.wantErr {
// The truncation below with uint32 cannot be defined at initialization (compiler error), performing it at runtime.
tt.want.(*tracesamplerprocessor).scaledSamplingRate = uint32(tt.cfg.SamplingPercentage * percentageScaleFactor)
}
got, err := NewTraceProcessor(tt.nextConsumer, tt.cfg)
if (err != nil) != tt.wantErr {
t.Errorf("NewTraceProcessor() error = %v, wantErr %v", err, tt.wantErr)
return
}
if !reflect.DeepEqual(got, tt.want) {
t.Errorf("NewTraceProcessor() = %v, want %v", got, tt.want)
}
})
}
}

// Test_tracesamplerprocessor_SamplingPercentageRange checks for different sampling rates and ensures
// that they are within acceptable deltas.
func Test_tracesamplerprocessor_SamplingPercentageRange(t *testing.T) {
tests := []struct {
name string
cfg TraceSamplerCfg
numBatches int
numTracesPerBatch int
acceptableDelta float64
}{
{
name: "random_sampling_tiny",
cfg: TraceSamplerCfg{
SamplingPercentage: 0.03,
},
numBatches: 1e5,
numTracesPerBatch: 2,
acceptableDelta: 0.01,
},
{
name: "random_sampling_small",
cfg: TraceSamplerCfg{
SamplingPercentage: 5,
},
numBatches: 1e5,
numTracesPerBatch: 2,
acceptableDelta: 0.01,
},
{
name: "random_sampling_medium",
cfg: TraceSamplerCfg{
SamplingPercentage: 50.0,
},
numBatches: 1e5,
numTracesPerBatch: 4,
acceptableDelta: 0.1,
},
{
name: "random_sampling_high",
cfg: TraceSamplerCfg{
SamplingPercentage: 90.0,
},
numBatches: 1e5,
numTracesPerBatch: 1,
acceptableDelta: 0.2,
},
{
name: "random_sampling_all",
cfg: TraceSamplerCfg{
SamplingPercentage: 100.0,
},
numBatches: 1e5,
numTracesPerBatch: 1,
acceptableDelta: 0.0,
},
}
const testSvcName = "test-svc"
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
sink := &exportertest.SinkTraceExporter{}
tsp, err := NewTraceProcessor(sink, tt.cfg)
if err != nil {
t.Errorf("error when creating tracesamplerprocessor: %v", err)
return
}
for _, td := range genRandomTestData(tt.numBatches, tt.numTracesPerBatch, testSvcName) {
if err := tsp.ConsumeTraceData(context.Background(), td); err != nil {
t.Errorf("tracesamplerprocessor.ConsumeTraceData() error = %v", err)
return
}
}
_, sampled := assertSampledData(t, sink.AllTraces(), testSvcName)
actualPercentageSamplingPercentage := float32(sampled) / float32(tt.numBatches*tt.numTracesPerBatch) * 100.0
delta := math.Abs(float64(actualPercentageSamplingPercentage - tt.cfg.SamplingPercentage))
if delta > tt.acceptableDelta {
t.Errorf(
"got %f percentage sampling rate, want %f (allowed delta is %f but got %f)",
actualPercentageSamplingPercentage,
tt.cfg.SamplingPercentage,
tt.acceptableDelta,
delta,
)
}
})
}
}

// Test_hash ensures that the hash function supports different key lengths even if in
// practice it is only expected to receive keys with length 16 (trace id length in OC proto).
func Test_hash(t *testing.T) {
// Statistically a random selection of such small number of keys should not result in
// collisions, but, of course it is possible that they happen, a different random source
// should avoid that.
r := rand.New(rand.NewSource(1))
fullKey := tracetranslator.UInt64ToByteTraceID(r.Uint64(), r.Uint64())
seen := make(map[uint32]bool)
for i := 1; i <= len(fullKey); i++ {
key := fullKey[:i]
hash := hash(key, 1)
if seen[hash] {
t.Fatal("Unexpected duplicated hash")
}
seen[hash] = true
}
}

// genRandomTestData generates a slice of data.TraceData with the numBatches elements which one with
// numTracesPerBatch spans (ie.: each span has a different trace ID). All spans belong to the specified
// serviceName.
func genRandomTestData(numBatches, numTracesPerBatch int, serviceName string) (tdd []data.TraceData) {
r := rand.New(rand.NewSource(1))

for i := 0; i < numBatches; i++ {
var spans []*tracepb.Span
for j := 0; j < numTracesPerBatch; j++ {
span := &tracepb.Span{
TraceId: tracetranslator.UInt64ToByteTraceID(r.Uint64(), r.Uint64()),
}
spans = append(spans, span)
}
td := data.TraceData{
Node: &commonpb.Node{
ServiceInfo: &commonpb.ServiceInfo{Name: serviceName},
},
Spans: spans,
}
tdd = append(tdd, td)
}

return tdd
}

// assertSampledData checks for no repeated traceIDs and counts the number of spans on the sampled data for
// the given service.
func assertSampledData(t *testing.T, sampled []data.TraceData, serviceName string) (traceIDs map[string]bool, spanCount int) {
traceIDs = make(map[string]bool)
for _, td := range sampled {
if processormetrics.ServiceNameForNode(td.Node) != serviceName {
continue
}
for _, span := range td.Spans {
spanCount++
key := string(span.TraceId)
if traceIDs[key] {
t.Errorf("same traceID used more than once %q", key)
return
}
traceIDs[key] = true
}
}
return
}

0 comments on commit 2a40f54

Please sign in to comment.