Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: trace sampling #72

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -15,15 +15,24 @@

import java.io.Serializable;
import java.time.Duration;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.function.Predicate;
import org.springframework.boot.context.properties.ConfigurationProperties;
import zipkin2.Span;
import zipkin2.storage.kafka.KafkaStorage;
import zipkin2.storage.kafka.KafkaStorageBuilder;
import zipkin2.storage.kafka.KafkaStorageBuilder.DependencyStorageBuilder;
import zipkin2.storage.kafka.KafkaStorageBuilder.SpanAggregationBuilder;
import zipkin2.storage.kafka.KafkaStorageBuilder.SpanPartitioningBuilder;
import zipkin2.storage.kafka.KafkaStorageBuilder.TraceStorageBuilder;
import zipkin2.storage.kafka.streams.samplers.CompositeTracePredicate;
import zipkin2.storage.kafka.streams.samplers.ErrorTracePredicate;
import zipkin2.storage.kafka.streams.samplers.SlowTracePredicate;
import zipkin2.storage.kafka.streams.samplers.TraceSamplingPredicate;

import static java.util.Arrays.asList;

@ConfigurationProperties("zipkin.storage.kafka")
public class ZipkinKafkaStorageProperties implements Serializable {
Expand All @@ -38,11 +47,13 @@ public class ZipkinKafkaStorageProperties implements Serializable {
private SpanAggregationProperties spanAggregation = new SpanAggregationProperties();
private TraceStorageProperties traceStorage = new TraceStorageProperties();
private DependencyStorageProperties dependencyStorage = new DependencyStorageProperties();
private TraceSamplingProperties traceSampling = new TraceSamplingProperties();

KafkaStorageBuilder toBuilder() {
KafkaStorageBuilder builder = KafkaStorage.newBuilder();
builder.spanPartitioningBuilder(spanPartitioning.toBuilder());
builder.spanAggregationBuilder(spanAggregation.toBuilder());
builder.spanAggregationBuilder(
spanAggregation.toBuilder().tracePredicate(traceSampling.tracePredicate()));
builder.traceStorageBuilder(traceStorage.toBuilder());
builder.dependencyStorageBuilder(dependencyStorage.toBuilder());
if (hostname != null) builder.hostname(hostname);
Expand Down Expand Up @@ -226,6 +237,35 @@ SpanAggregationBuilder toBuilder() {
}
}

static class TraceSamplingProperties {
private long maxResponseTime = 1_000_000L;
private float sampleRate = 0.1f;
private boolean enabled = false;

public void setMaxResponseTime(long maxResponseTime) {
this.maxResponseTime = maxResponseTime;
}

public void setSampleRate(float sampleRate) {
this.sampleRate = sampleRate;
}

public void setEnabled(boolean enabled) {
this.enabled = enabled;
}

Predicate<Collection<Span>> tracePredicate() {
if (enabled) {
return new CompositeTracePredicate(asList(
new ErrorTracePredicate(),
new SlowTracePredicate(maxResponseTime),
new TraceSamplingPredicate(sampleRate))
);
}
return spans -> true;
}
}

static class TraceStorageProperties {
private Boolean enabled;
private String spansTopic;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,8 @@ public static KafkaStorageBuilder newBuilder() {
builder.spanAggregation.traceTopic,
builder.spanAggregation.dependencyTopic,
builder.spanAggregation.traceTimeout,
builder.spanAggregation.enabled).get();
builder.spanAggregation.enabled,
builder.spanAggregation.tracePredicate).get();
traceStoreTopology = new TraceStorageTopology(
builder.traceStorage.spansTopic,
autocompleteKeys,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,17 +15,20 @@

import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.function.BiFunction;
import java.util.function.Predicate;
import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.Serdes.ByteArraySerde;
import org.apache.kafka.common.serialization.Serdes.StringSerde;
import org.apache.kafka.common.serialization.StringSerializer;
import org.apache.kafka.streams.StreamsConfig;
import zipkin2.Span;
import zipkin2.storage.StorageComponent;

import static zipkin2.storage.kafka.KafkaStorage.HTTP_PATH_PREFIX;
Expand Down Expand Up @@ -230,6 +233,7 @@ public final SpanPartitioningBuilder overrides(Map<String, ?> overrides) {
}

public static class SpanAggregationBuilder {
Predicate<Collection<Span>> tracePredicate = spans -> true;
boolean enabled = true;
Duration traceTimeout = Duration.ofMinutes(1);
String spansTopic = "zipkin-spans";
Expand All @@ -246,6 +250,14 @@ public SpanAggregationBuilder() {
streamConfig.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE);
}

/**
* Filter traces based on sampling rules.
*/
public SpanAggregationBuilder tracePredicate(Predicate<Collection<Span>> tracePredicate) {
this.tracePredicate = tracePredicate;
return this;
}

/**
* Enable aggregation stream application to run. When disabled spans will not be consumed to
* produce traces and dependencies.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,9 @@

import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.function.Predicate;
import java.util.function.Supplier;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.streams.StreamsBuilder;
Expand Down Expand Up @@ -54,6 +56,7 @@ public final class SpanAggregationTopology implements Supplier<Topology> {
// SerDes
final SpansSerde spansSerde;
final DependencyLinkSerde dependencyLinkSerde;
final Predicate<Collection<Span>> tracePredicate;

public SpanAggregationTopology(
String spansTopic,
Expand All @@ -62,11 +65,17 @@ public SpanAggregationTopology(
Duration traceTimeout,
boolean aggregationEnabled
) {
this(spansTopic, traceTopic, dependencyTopic, traceTimeout, aggregationEnabled, spans -> true);
}

public SpanAggregationTopology(String spansTopic, String traceTopic, String dependencyTopic,
Duration traceTimeout, boolean aggregationEnabled, Predicate<Collection<Span>> tracePredicate) {
this.spansTopic = spansTopic;
this.traceTopic = traceTopic;
this.dependencyTopic = dependencyTopic;
this.traceTimeout = traceTimeout;
this.aggregationEnabled = aggregationEnabled;
this.tracePredicate = tracePredicate;
spansSerde = new SpansSerde();
dependencyLinkSerde = new DependencyLinkSerde();
}
Expand All @@ -92,6 +101,7 @@ public SpanAggregationTopology(
// hold until a new record tells that a window is closed and we can process it further
.suppress(untilWindowCloses(unbounded()))
.toStream()
.filter((traceId, spans) -> tracePredicate.test(spans))
.selectKey((windowed, spans) -> windowed.key());
// Downstream to traces topic
tracesStream.to(traceTopic, Produced.with(Serdes.String(), spansSerde));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,37 @@
/*
* Copyright 2019 The OpenZipkin 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 zipkin2.storage.kafka.streams.samplers;

import java.util.Collection;
import java.util.function.Predicate;
import zipkin2.Span;

public class CompositeTracePredicate implements Predicate<Collection<Span>> {
private final Collection<Predicate<Collection<Span>>> predicates;

public CompositeTracePredicate(Collection<Predicate<Collection<Span>>> predicates) {
this.predicates = predicates;
}

@Override public boolean test(Collection<Span> spans) {
for (Predicate<Collection<Span>> predicate : predicates) {
if (predicate.test(spans)) {
return true;
}
}
return false;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright 2019 The OpenZipkin 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 zipkin2.storage.kafka.streams.samplers;

import java.util.Collection;
import java.util.function.Predicate;
import zipkin2.Span;

public class ErrorTracePredicate implements Predicate<Collection<Span>> {
@Override public boolean test(Collection<Span> spans) {
return spans.stream().anyMatch(s -> s.tags().containsKey("error"));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* Copyright 2019 The OpenZipkin 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 zipkin2.storage.kafka.streams.samplers;

import java.util.Collection;
import java.util.function.Predicate;
import zipkin2.Span;

public class SlowTracePredicate implements Predicate<Collection<Span>> {
private final long maxResponseTime;

public SlowTracePredicate(long maxResponseTime) {
this.maxResponseTime = maxResponseTime;
}

@Override public boolean test(Collection<Span> spans) {
return spans.stream()
.anyMatch(span -> span.durationAsLong() >= maxResponseTime);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
/*
* Copyright 2019 The OpenZipkin 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 zipkin2.storage.kafka.streams.samplers;

import java.util.Collection;
import java.util.function.Predicate;
import zipkin2.Span;
import zipkin2.internal.HexCodec;

public class TraceSamplingPredicate implements Predicate<Collection<Span>> {
private final float rate;

public TraceSamplingPredicate(float rate) {
if (rate < 0.f || rate > 1.f)
throw new IllegalArgumentException("rate should be between 0 and 1: was " + rate);

this.rate = rate;
}

@Override public boolean test(Collection<Span> spans) {
return spans.stream()
.anyMatch(span -> {
if (Boolean.TRUE.equals(span.debug())) return true;
long traceId = HexCodec.lowerHexToUnsignedLong(span.traceId());
// The absolute value of Long.MIN_VALUE is larger than a long, so Math.abs returns identity.
// This converts to MAX_VALUE to avoid always dropping when traceId == Long.MIN_VALUE
long t = traceId == Long.MIN_VALUE ? Long.MAX_VALUE : Math.abs(traceId);
return t <= (long) (Long.MAX_VALUE * rate);
});
}
}
Loading