From 95d5ea019b8f60d670ee2bb967b101ec19db1c88 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Fri, 13 Oct 2023 01:01:09 +0530 Subject: [PATCH 01/13] Support service correlation for mirroring spans --- .../constants/EnrichedSpanConstants.java | 2 + .../constants/utils/EnrichedSpanUtils.java | 2 +- raw-spans-grouper/helm/values.yaml | 5 + .../raw-spans-grouper/build.gradle.kts | 1 + .../RawSpanGrouperConstants.java | 2 + .../core/rawspansgrouper/RawSpansGrouper.java | 15 +- .../rawspansgrouper/RawSpansProcessor.java | 132 +++++++++++++++++- .../rawspansgrouper/TraceEmitPunctuator.java | 27 +--- .../utils/RawSpansGrouperUtils.java | 37 +++++ .../utils/RawSpansGrouperUtilsTest.java | 31 ++++ .../http/HttpSemanticConventionUtils.java | 25 +++- .../avro/IpResolutionStateStoreValue.avdl | 6 + .../IpResolutionExitSpanIdentity.avdl | 10 ++ .../span/OTelSpanSemanticConventions.java | 3 + 14 files changed, 266 insertions(+), 32 deletions(-) create mode 100644 raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java create mode 100644 raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java create mode 100644 span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl create mode 100644 span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl diff --git a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java index 58fcddbe0..3a1931a99 100644 --- a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java +++ b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java @@ -18,6 +18,8 @@ public class EnrichedSpanConstants { public static final String GRPC_REQUEST_URL = "grpc.request.url"; public static final String GRPC_REQUEST_ENDPOINT = "grpc.request.endpoint"; public static final String DROP_TRACE_ATTRIBUTE = "drop.trace"; + public static final String CALLER_SERVICE_ID = "CALLER_SERVICE_ID"; + public static final String CALLER_SERVICE_NAME = "CALLER_SERVICE_NAME"; /** * Returns the constant value for the given Enum. diff --git a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java index a55f63ee0..38f0cffd8 100644 --- a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java +++ b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java @@ -350,7 +350,7 @@ public static Optional getFullHttpUrl(Event event) { } public static Optional getDestinationIpAddress(Event event) { - return HttpSemanticConventionUtils.getDestinationIpAddress(event); + return HttpSemanticConventionUtils.getPeerIpAddress(event); } public static Optional getPath(Event event) { diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 028f87f4b..3c1605692 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -169,6 +169,11 @@ kafka-topic-creator: partitions: 8 configs: cleanup.policy: "[compact, delete]" + raw-spans-to-structured-traces-grouping-job-mirroring-exit-spans-state-store-changelog: + replicationFactor: 3 + partitions: 8 + configs: + cleanup.policy: "[compact, delete]" zookeeper: address: zookeeper:2181 diff --git a/raw-spans-grouper/raw-spans-grouper/build.gradle.kts b/raw-spans-grouper/raw-spans-grouper/build.gradle.kts index e3f795799..afcd8437c 100644 --- a/raw-spans-grouper/raw-spans-grouper/build.gradle.kts +++ b/raw-spans-grouper/raw-spans-grouper/build.gradle.kts @@ -33,6 +33,7 @@ dependencies { because("https://snyk.io/vuln/SNYK-JAVA-ORGGLASSFISHJERSEYCORE-1255637") } implementation(project(":span-normalizer:span-normalizer-api")) + implementation(project(":semantic-convention-utils")) implementation(libs.hypertrace.data.model) implementation(libs.hypertrace.serviceFramework.framework) implementation(libs.hypertrace.serviceFramework.metrics) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index 244782024..bf89f3baf 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -8,6 +8,7 @@ public class RawSpanGrouperConstants { public static final String RAW_SPANS_GROUPER_JOB_CONFIG = "raw-spans-grouper-job-config"; public static final String SPAN_STATE_STORE_NAME = "span-data-store"; public static final String TRACE_STATE_STORE = "trace-state-store"; + public static final String MIRRORING_EXIT_SPANS_STATE_STORE = "mirroring-exit-spans-state-store"; public static final String OUTPUT_TOPIC_PRODUCER = "output-topic-producer"; public static final String SPANS_PER_TRACE_METRIC = "spans_per_trace"; public static final String TRACE_CREATION_TIME = "trace.creation.time"; @@ -17,4 +18,5 @@ public class RawSpanGrouperConstants { public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; public static final String TRUNCATED_TRACES_COUNTER = "hypertrace.truncated.traces"; + public static final String CALLER_SERVICE_NAME = "caller.service.name"; } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java index d70d8ba9f..3ebd1d674 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java @@ -1,6 +1,7 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.MIRRORING_EXIT_SPANS_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; @@ -26,6 +27,8 @@ import org.hypertrace.core.kafkastreams.framework.partitioner.GroupPartitionerBuilder; import org.hypertrace.core.kafkastreams.framework.partitioner.KeyHashPartitioner; import org.hypertrace.core.serviceframework.config.ConfigClient; +import org.hypertrace.core.spannormalizer.IpResolutionExitSpanIdentity; +import org.hypertrace.core.spannormalizer.IpResolutionStateStoreValue; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; import org.hypertrace.core.spannormalizer.TraceState; @@ -72,8 +75,17 @@ public StreamsBuilder buildTopology( Stores.persistentKeyValueStore(SPAN_STATE_STORE_NAME), keySerde, valueSerde) .withCachingEnabled(); + StoreBuilder> + mirroringExitSpansStoreBuilder = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(MIRRORING_EXIT_SPANS_STATE_STORE), + keySerde, + valueSerde) + .withCachingEnabled(); + streamsBuilder.addStateStore(spanStoreBuilder); streamsBuilder.addStateStore(traceStateStoreBuilder); + streamsBuilder.addStateStore(mirroringExitSpansStoreBuilder); StreamPartitioner groupPartitioner = new GroupPartitionerBuilder() @@ -93,7 +105,8 @@ public StreamsBuilder buildTopology( RawSpansProcessor::new, Named.as(RawSpansProcessor.class.getSimpleName()), SPAN_STATE_STORE_NAME, - TRACE_STATE_STORE) + TRACE_STATE_STORE, + MIRRORING_EXIT_SPANS_STATE_STORE) .to(outputTopic, outputTopicProducer); return streamsBuilder; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 1dc710bcc..981baae82 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -1,10 +1,12 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.datamodel.shared.AvroBuilderCache.fastNewBuilder; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.CALLER_SERVICE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DROPPED_SPANS_COUNTER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INFLIGHT_TRACE_MAX_SPAN_COUNT; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.MIRRORING_EXIT_SPANS_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY; @@ -21,6 +23,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; @@ -33,13 +37,24 @@ import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; +import org.hypertrace.core.datamodel.AttributeValue; +import org.hypertrace.core.datamodel.Attributes; +import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; +import org.hypertrace.core.datamodel.Timestamps; import org.hypertrace.core.datamodel.shared.HexUtils; +import org.hypertrace.core.datamodel.shared.trace.AttributeValueCreator; +import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; +import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; +import org.hypertrace.core.spannormalizer.IpResolutionExitSpanIdentity; +import org.hypertrace.core.spannormalizer.IpResolutionStateStoreValue; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; import org.hypertrace.core.spannormalizer.TraceState; +import org.hypertrace.semantic.convention.utils.http.HttpSemanticConventionUtils; +import org.hypertrace.semantic.convention.utils.span.SpanSemanticConventionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,11 +71,15 @@ public class RawSpansProcessor private static final Logger logger = LoggerFactory.getLogger(RawSpansProcessor.class); private static final String PROCESSING_LATENCY_TIMER = "hypertrace.rawspansgrouper.processing.latency"; + private static final String MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG = + "mirroring.span.attribute.name"; private static final ConcurrentMap tenantToSpansGroupingTimer = new ConcurrentHashMap<>(); private ProcessorContext context; private KeyValueStore spanStore; private KeyValueStore traceStateStore; + private KeyValueStore + mirroringExitSpansStateStore; private long groupingWindowTimeoutMs; private To outputTopic; private double dataflowSamplingPercent = -1; @@ -74,6 +93,8 @@ public class RawSpansProcessor // counter for number of truncated traces per tenant private static final ConcurrentMap truncatedTracesCounter = new ConcurrentHashMap<>(); + private String mirroringSpanAttributeName; + private RawSpansGrouperUtils rawSpansGrouperUtils; @Override public void init(ProcessorContext context) { @@ -82,7 +103,14 @@ public void init(ProcessorContext context) { (KeyValueStore) context.getStateStore(SPAN_STATE_STORE_NAME); this.traceStateStore = (KeyValueStore) context.getStateStore(TRACE_STATE_STORE); + this.mirroringExitSpansStateStore = + (KeyValueStore) + context.getStateStore(MIRRORING_EXIT_SPANS_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); + this.mirroringSpanAttributeName = + jobConfig.hasPath(MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG) + ? jobConfig.getString(MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG) + : null; this.groupingWindowTimeoutMs = jobConfig.getLong(SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY) * 1000; @@ -91,14 +119,14 @@ public void init(ProcessorContext context) { && jobConfig.getDouble(DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY) <= 100) { this.dataflowSamplingPercent = jobConfig.getDouble(DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY); } + this.rawSpansGrouperUtils = new RawSpansGrouperUtils(dataflowSamplingPercent); if (jobConfig.hasPath(INFLIGHT_TRACE_MAX_SPAN_COUNT)) { Config subConfig = jobConfig.getConfig(INFLIGHT_TRACE_MAX_SPAN_COUNT); - subConfig.entrySet().stream() + subConfig + .entrySet() .forEach( - (entry) -> { - maxSpanCountMap.put(entry.getKey(), subConfig.getLong(entry.getKey())); - }); + (entry) -> maxSpanCountMap.put(entry.getKey(), subConfig.getLong(entry.getKey()))); } if (jobConfig.hasPath(DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT)) { @@ -114,15 +142,20 @@ public KeyValue transform(TraceIdentity key, Raw long currentTimeMs = System.currentTimeMillis(); TraceState traceState = traceStateStore.get(key); - boolean firstEntry = (traceState == null); - if (shouldDropSpan(key, traceState)) { return null; } + Event event = value.getEvent(); + if (isMirroringSpan(event)) { + processMirroringSpan(key, value, traceState, currentTimeMs); + return null; + } + + boolean firstEntry = (traceState == null); String tenantId = key.getTenantId(); ByteBuffer traceId = value.getTraceId(); - ByteBuffer spanId = value.getEvent().getEventId(); + ByteBuffer spanId = event.getEventId(); spanStore.put(new SpanIdentity(tenantId, traceId, spanId), value); /* @@ -169,6 +202,91 @@ public KeyValue transform(TraceIdentity key, Raw return null; } + private void processMirroringSpan( + TraceIdentity key, RawSpan value, TraceState traceState, long currentTimeMs) { + Event event = value.getEvent(); + String tenantId = key.getTenantId(); + ByteBuffer traceId = value.getTraceId(); + boolean firstEntry = (traceState == null); + final Optional maybeEnvironment = + HttpSemanticConventionUtils.getEnvironmentForSpan(event); + + if (SpanSemanticConventionUtils.isClientSpanForOCFormat( + event.getAttributes().getAttributeMap())) { + final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); + final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); + final Optional maybePeerPort = HttpSemanticConventionUtils.getPeerPort(event); + if (maybeEnvironment.isPresent() + && maybeHostAddr.isPresent() + && maybePeerAddr.isPresent() + && maybePeerPort.isPresent()) { + final String serviceName = event.getServiceName(); + mirroringExitSpansStateStore.put( + IpResolutionExitSpanIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(maybeEnvironment.get()) + .setHostAddr(maybeHostAddr.get()) + .setPeerAddr(maybePeerAddr.get()) + .setPeerPort(maybePeerPort.get()) + .build(), + IpResolutionStateStoreValue.newBuilder().setServiceName(serviceName).build()); + } + } else { + final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); + final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); + final Optional maybeHostPort = HttpSemanticConventionUtils.getHostPort(event); + if (maybeEnvironment.isPresent() + && maybePeerAddr.isPresent() + && maybeHostAddr.isPresent() + && maybeHostPort.isPresent()) { + final IpResolutionExitSpanIdentity ipResolutionIdentity = + IpResolutionExitSpanIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(maybeEnvironment.get()) + .setHostAddr(maybePeerAddr.get()) + .setPeerAddr(maybeHostAddr.get()) + .setPeerPort(maybeHostPort.get()) + .build(); + final IpResolutionStateStoreValue ipResolutionStateStoreValue = + mirroringExitSpansStateStore.get(ipResolutionIdentity); + if (Objects.nonNull(ipResolutionStateStoreValue)) { + event + .getAttributes() + .getAttributeMap() + .put( + CALLER_SERVICE_NAME, + AttributeValueCreator.create(ipResolutionStateStoreValue.getServiceName())); + } + } + } + + Timestamps timestamps = + rawSpansGrouperUtils.trackEndToEndLatencyTimestamps( + currentTimeMs, firstEntry ? currentTimeMs : traceState.getTraceStartTimestamp()); + StructuredTrace trace = + StructuredTraceBuilder.buildStructuredTraceFromRawSpans( + List.of(value), traceId, tenantId, timestamps); + context.forward(key, trace, outputTopic); + } + + private boolean isMirroringSpan(Event event) { + final Attributes attributes = event.getAttributes(); + if (Objects.isNull(attributes)) { + return false; + } + + final Map attributeMap = attributes.getAttributeMap(); + if (Objects.isNull(attributeMap)) { + return false; + } + + final String value = + attributeMap + .getOrDefault(this.mirroringSpanAttributeName, AttributeValue.newBuilder().build()) + .getValue(); + return Objects.nonNull(value) && Boolean.parseBoolean(value); + } + private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { int inFlightSpansPerTrace = traceState != null ? traceState.getSpanIds().size() : Integer.MIN_VALUE; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java index 0c0cae52a..e73420839 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java @@ -1,7 +1,6 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPANS_PER_TRACE_METRIC; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_CREATION_TIME; import com.google.common.util.concurrent.RateLimiter; import io.micrometer.core.instrument.Counter; @@ -28,11 +27,11 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; -import org.hypertrace.core.datamodel.TimestampRecord; import org.hypertrace.core.datamodel.Timestamps; import org.hypertrace.core.datamodel.shared.DataflowMetricUtils; import org.hypertrace.core.datamodel.shared.HexUtils; import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; +import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -69,14 +68,13 @@ class TraceEmitPunctuator implements Punctuator { "hypertrace.rawspansgrouper.trace.with.duplicate.spans"; private static final ConcurrentMap tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); - - private final double dataflowSamplingPercent; private final TraceIdentity key; private final ProcessorContext context; private final KeyValueStore spanStore; private final KeyValueStore traceStateStore; private final To outputTopicProducer; private final long groupingWindowTimeoutMs; + private final RawSpansGrouperUtils rawSpansGrouperUtils; private Cancellable cancellable; TraceEmitPunctuator( @@ -93,7 +91,7 @@ class TraceEmitPunctuator implements Punctuator { this.traceStateStore = traceStateStore; this.outputTopicProducer = outputTopicProducer; this.groupingWindowTimeoutMs = groupingWindowTimeoutMs; - this.dataflowSamplingPercent = dataflowSamplingPercent; + this.rawSpansGrouperUtils = new RawSpansGrouperUtils(dataflowSamplingPercent); } public void setCancellable(Cancellable cancellable) { @@ -167,7 +165,8 @@ public void punctuate(long timestamp) { recordSpansPerTrace(rawSpanList.size(), List.of(Tag.of("tenant_id", tenantId))); Timestamps timestamps = - trackEndToEndLatencyTimestamps(timestamp, traceState.getTraceStartTimestamp()); + rawSpansGrouperUtils.trackEndToEndLatencyTimestamps( + timestamp, traceState.getTraceStartTimestamp()); StructuredTrace trace = StructuredTraceBuilder.buildStructuredTraceFromRawSpans( rawSpanList, traceId, tenantId, timestamps); @@ -235,22 +234,6 @@ public void punctuate(long timestamp) { } } - private Timestamps trackEndToEndLatencyTimestamps( - long currentTimestamp, long firstSpanTimestamp) { - Timestamps timestamps = null; - if (!(Math.random() * 100 <= dataflowSamplingPercent)) { - spansGrouperArrivalLagTimer.record( - currentTimestamp - firstSpanTimestamp, TimeUnit.MILLISECONDS); - Map records = new HashMap<>(); - records.put( - DataflowMetricUtils.SPAN_ARRIVAL_TIME, - new TimestampRecord(DataflowMetricUtils.SPAN_ARRIVAL_TIME, firstSpanTimestamp)); - records.put(TRACE_CREATION_TIME, new TimestampRecord(TRACE_CREATION_TIME, currentTimestamp)); - timestamps = new Timestamps(records); - } - return timestamps; - } - private void recordSpansPerTrace(double count, Iterable tags) { DistributionSummary summary = DistributionSummary.builder(SPANS_PER_TRACE_METRIC) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java new file mode 100644 index 000000000..795b39f04 --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java @@ -0,0 +1,37 @@ +package org.hypertrace.core.rawspansgrouper.utils; + +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_CREATION_TIME; + +import io.micrometer.core.instrument.Timer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.hypertrace.core.datamodel.TimestampRecord; +import org.hypertrace.core.datamodel.Timestamps; +import org.hypertrace.core.datamodel.shared.DataflowMetricUtils; +import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; + +public class RawSpansGrouperUtils { + private static final Timer spansGrouperArrivalLagTimer = + PlatformMetricsRegistry.registerTimer(DataflowMetricUtils.ARRIVAL_LAG, new HashMap<>()); + private final double dataflowSamplingPercent; + + public RawSpansGrouperUtils(double dataflowSamplingPercent) { + this.dataflowSamplingPercent = dataflowSamplingPercent; + } + + public Timestamps trackEndToEndLatencyTimestamps(long currentTimestamp, long firstSpanTimestamp) { + Timestamps timestamps = null; + if (!(Math.random() * 100 <= dataflowSamplingPercent)) { + spansGrouperArrivalLagTimer.record( + currentTimestamp - firstSpanTimestamp, TimeUnit.MILLISECONDS); + Map records = new HashMap<>(); + records.put( + DataflowMetricUtils.SPAN_ARRIVAL_TIME, + new TimestampRecord(DataflowMetricUtils.SPAN_ARRIVAL_TIME, firstSpanTimestamp)); + records.put(TRACE_CREATION_TIME, new TimestampRecord(TRACE_CREATION_TIME, currentTimestamp)); + timestamps = new Timestamps(records); + } + return timestamps; + } +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java new file mode 100644 index 000000000..ea2a26edb --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java @@ -0,0 +1,31 @@ +package org.hypertrace.core.rawspansgrouper.utils; + +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_CREATION_TIME; + +import java.util.Map; +import org.hypertrace.core.datamodel.TimestampRecord; +import org.hypertrace.core.datamodel.Timestamps; +import org.hypertrace.core.datamodel.shared.DataflowMetricUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class RawSpansGrouperUtilsTest { + + @Test + void testTrackEndToEndLatencyTimestamps() { + RawSpansGrouperUtils rawSpansGrouperUtils = new RawSpansGrouperUtils(100); + Timestamps timestamps = rawSpansGrouperUtils.trackEndToEndLatencyTimestamps(123L, 123L); + Assertions.assertNull(timestamps); + + rawSpansGrouperUtils = new RawSpansGrouperUtils(0); + timestamps = rawSpansGrouperUtils.trackEndToEndLatencyTimestamps(150L, 100L); + Assertions.assertEquals( + new Timestamps( + Map.of( + DataflowMetricUtils.SPAN_ARRIVAL_TIME, + new TimestampRecord(DataflowMetricUtils.SPAN_ARRIVAL_TIME, 100L), + TRACE_CREATION_TIME, + new TimestampRecord(TRACE_CREATION_TIME, 150L))), + timestamps); + } +} diff --git a/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java b/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java index ec981e553..3b617f0c6 100644 --- a/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java +++ b/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java @@ -212,7 +212,7 @@ public static Optional getFullHttpUrl(Event event) { : getValidHttpUrl(event).map(AttributeValue::getValue); } - public static Optional getDestinationIpAddress(Event event) { + public static Optional getPeerIpAddress(Event event) { return Optional.ofNullable( SpanAttributeUtils.getStringAttribute( event, OTelSpanSemanticConventions.NET_SOCK_PEER_ADDR.getValue())) @@ -223,6 +223,29 @@ public static Optional getDestinationIpAddress(Event event) { event, OTelSpanSemanticConventions.NET_PEER_IP.getValue()))); } + public static Optional getPeerPort(Event event) { + return Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_SOCK_PEER_PORT.getValue())) + .or( + () -> + Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_PEER_PORT.getValue()))); + } + + public static Optional getHostIpAddress(Event event) { + return Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_SOCK_HOST_ADDR.getValue())); + } + + public static Optional getHostPort(Event event) { + return Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_SOCK_HOST_PORT.getValue())); + } + public static Optional getEnvironmentForSpan(Event event) { return Optional.ofNullable( SpanAttributeUtils.getStringAttribute( diff --git a/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl b/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl new file mode 100644 index 000000000..76644a2eb --- /dev/null +++ b/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl @@ -0,0 +1,6 @@ +@namespace("org.hypertrace.core.spannormalizer") +protocol IpResolutionStateStoreValueProtocol { + record IpResolutionStateStoreValue { + string service_name; + } +} diff --git a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl new file mode 100644 index 000000000..5b147aa31 --- /dev/null +++ b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl @@ -0,0 +1,10 @@ +@namespace("org.hypertrace.core.spannormalizer") +protocol IpResolutionExitSpanIdentityProtocol { + record IpResolutionExitSpanIdentity { + string tenant_id; + string environment; + string host_addr; + string peer_addr; + string peer_port; + } +} diff --git a/span-normalizer/span-normalizer-constants/src/main/java/org/hypertrace/core/semantic/convention/constants/span/OTelSpanSemanticConventions.java b/span-normalizer/span-normalizer-constants/src/main/java/org/hypertrace/core/semantic/convention/constants/span/OTelSpanSemanticConventions.java index e68a91d35..c29fa0834 100644 --- a/span-normalizer/span-normalizer-constants/src/main/java/org/hypertrace/core/semantic/convention/constants/span/OTelSpanSemanticConventions.java +++ b/span-normalizer/span-normalizer-constants/src/main/java/org/hypertrace/core/semantic/convention/constants/span/OTelSpanSemanticConventions.java @@ -9,6 +9,9 @@ public enum OTelSpanSemanticConventions { NET_PEER_PORT("net.peer.port"), NET_PEER_NAME("net.peer.name"), NET_SOCK_PEER_ADDR("net.sock.peer.addr"), + NET_SOCK_PEER_PORT("net.sock.peer.port"), + NET_SOCK_HOST_ADDR("net.sock.host.addr"), + NET_SOCK_HOST_PORT("net.sock.host.port"), NET_TRANSPORT("net.transport"), HTTP_CLIENT_IP("http.client_ip"); From 25bcf21983ebb76c0e5c39edc21c88183e6fe90a Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Fri, 13 Oct 2023 02:25:52 +0530 Subject: [PATCH 02/13] Added test --- .../rawspansgrouper/RawSpansGrouperTest.java | 173 ++++++++++++++++++ .../raw-spans-grouper/application.conf | 2 + 2 files changed, 175 insertions(+) diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index 3d96e9b53..ebf0b4892 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -22,6 +22,8 @@ import org.apache.kafka.streams.TestOutputTopic; import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.test.TestRecord; +import org.hypertrace.core.datamodel.AttributeValue; +import org.hypertrace.core.datamodel.Attributes; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -274,6 +276,137 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertEquals(6, testRecord.getValue().getEventList().size()); } + @Test + @SetEnvironmentVariable(key = "SERVICE_NAME", value = "raw-spans-grouper") + void testMirroringSpansGrouping(@TempDir Path tempDir) { + File file = tempDir.resolve("state").toFile(); + + RawSpansGrouper underTest = new RawSpansGrouper(ConfigClientFactory.getClient()); + Config config = + ConfigFactory.parseURL( + getClass().getClassLoader().getResource("configs/raw-spans-grouper/application.conf")); + + Map baseProps = underTest.getBaseStreamsConfig(); + Map streamsProps = underTest.getStreamsConfig(config); + baseProps.forEach(streamsProps::put); + Map mergedProps = streamsProps; + + mergedProps.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); + mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); + mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); + + StreamsBuilder streamsBuilder = + underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); + + Properties props = new Properties(); + mergedProps.forEach(props::put); + + Serde defaultValueSerde = new StreamsConfig(mergedProps).defaultValueSerde(); + + Serde traceIdentitySerde = new StreamsConfig(mergedProps).defaultKeySerde(); + + TopologyTestDriver td = new TopologyTestDriver(streamsBuilder.build(), props); + TestInputTopic inputTopic = + td.createInputTopic( + config.getString(RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY), + traceIdentitySerde.serializer(), + defaultValueSerde.serializer()); + + TestOutputTopic outputTopic = + td.createOutputTopic( + config.getString(RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY), + traceIdentitySerde.deserializer(), + defaultValueSerde.deserializer()); + + String hostAddr1 = "1.2.3.4"; + String hostAddr2 = "1.2.3.5"; + String hostAddr3 = "1.2.3.6"; + String hostPort2 = "5000"; + String hostPort3 = "6000"; + String service1 = "service1"; + String service2 = "service2"; + String service3 = "service3"; + + String tenantId = "tenant1"; + RawSpan span1 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-1", tenantId, service1, "client", hostAddr1, "1234", hostAddr2, + hostPort2)) + .build(); + RawSpan span2 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-2".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-2", tenantId, service3, "server", hostAddr3, hostPort3, hostAddr2, + "5678")) + .build(); + RawSpan span3 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-3", tenantId, service2, "server", hostAddr2, hostPort2, hostAddr1, + "1234")) + .build(); + RawSpan span4 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-4".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-4", tenantId, service2, "client", hostAddr2, "5678", hostAddr3, + hostPort3)) + .build(); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); + StructuredTrace trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span1.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span2.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span3); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + Event event = span3.getEvent(); + Map attributeMap = + new HashMap<>(event.getAttributes().getAttributeMap()); + attributeMap.put(RawSpanGrouperConstants.CALLER_SERVICE_NAME, createAttribute(service1)); + event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + assertTrue( + trace + .getEventList() + .get(0) + .getAttributes() + .getAttributeMap() + .containsKey(RawSpanGrouperConstants.CALLER_SERVICE_NAME)); + assertEquals(event, trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span4.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + event = span2.getEvent(); + attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); + attributeMap.put(RawSpanGrouperConstants.CALLER_SERVICE_NAME, createAttribute(service2)); + event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + assertEquals(event, trace.getEventList().get(0)); + } + private Event createEvent(String eventId, String tenantId) { return Event.newBuilder() .setCustomerId(tenantId) @@ -282,6 +415,46 @@ private Event createEvent(String eventId, String tenantId) { .build(); } + private Event createMirroringEvent( + String eventId, + String tenantId, + String service, + String spanKind, + String hostAddr, + String hostPort, + String peerAddr, + String peerPort) { + return Event.newBuilder() + .setCustomerId(tenantId) + .setServiceName(service) + .setEventId(ByteBuffer.wrap(eventId.getBytes())) + .setStartTimeMillis(System.currentTimeMillis()) + .setAttributes( + Attributes.newBuilder() + .setAttributeMap( + Map.of( + "mirroring.enabled", + createAttribute("true"), + "deployment.environment", + createAttribute("environment"), + "span.kind", + createAttribute(spanKind), + "net.sock.host.addr", + createAttribute(hostAddr), + "net.sock.host.port", + createAttribute(hostPort), + "net.sock.peer.addr", + createAttribute(peerAddr), + "net.sock.peer.port", + createAttribute(peerPort))) + .build()) + .build(); + } + + private AttributeValue createAttribute(String value) { + return AttributeValue.newBuilder().setValue(value).build(); + } + private TraceIdentity createTraceIdentity(String tenantId, String traceId) { return TraceIdentity.newBuilder() .setTenantId(tenantId) diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index 5284b3714..04947553e 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -44,3 +44,5 @@ span.groupby.session.window.interval = 5 span.groupby.session.window.graceperiod.ms = 100 dataflow.metriccollection.sampling.percent = 10.0 + +mirroring.span.attribute.name = "mirroring.enabled" From c7885741e7581bf8458dd24e974ec1e2ab2b15fe Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Fri, 13 Oct 2023 20:29:32 +0530 Subject: [PATCH 03/13] Updated caller to peer --- .../enrichedspan/constants/EnrichedSpanConstants.java | 4 ++-- .../core/rawspansgrouper/RawSpanGrouperConstants.java | 2 +- .../core/rawspansgrouper/RawSpansProcessor.java | 4 ++-- .../core/rawspansgrouper/RawSpansGrouperTest.java | 11 ++--------- 4 files changed, 7 insertions(+), 14 deletions(-) diff --git a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java index 3a1931a99..5a33d8c03 100644 --- a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java +++ b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/EnrichedSpanConstants.java @@ -18,8 +18,8 @@ public class EnrichedSpanConstants { public static final String GRPC_REQUEST_URL = "grpc.request.url"; public static final String GRPC_REQUEST_ENDPOINT = "grpc.request.endpoint"; public static final String DROP_TRACE_ATTRIBUTE = "drop.trace"; - public static final String CALLER_SERVICE_ID = "CALLER_SERVICE_ID"; - public static final String CALLER_SERVICE_NAME = "CALLER_SERVICE_NAME"; + public static final String PEER_SERVICE_NAME = "PEER_SERVICE_NAME"; + public static final String PEER_SERVICE_ID = "PEER_SERVICE_ID"; /** * Returns the constant value for the given Enum. diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index bf89f3baf..0012098c0 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -18,5 +18,5 @@ public class RawSpanGrouperConstants { public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; public static final String TRUNCATED_TRACES_COUNTER = "hypertrace.truncated.traces"; - public static final String CALLER_SERVICE_NAME = "caller.service.name"; + public static final String PEER_SERVICE_NAME = "peer.service.name"; } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 981baae82..e11c07461 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -1,13 +1,13 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.datamodel.shared.AvroBuilderCache.fastNewBuilder; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.CALLER_SERVICE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DROPPED_SPANS_COUNTER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.MIRRORING_EXIT_SPANS_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_STATE_STORE_NAME; @@ -254,7 +254,7 @@ private void processMirroringSpan( .getAttributes() .getAttributeMap() .put( - CALLER_SERVICE_NAME, + PEER_SERVICE_NAME, AttributeValueCreator.create(ipResolutionStateStoreValue.getServiceName())); } } diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index ebf0b4892..1c832f374 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -381,15 +381,8 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { Event event = span3.getEvent(); Map attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.CALLER_SERVICE_NAME, createAttribute(service1)); + attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); - assertTrue( - trace - .getEventList() - .get(0) - .getAttributes() - .getAttributeMap() - .containsKey(RawSpanGrouperConstants.CALLER_SERVICE_NAME)); assertEquals(event, trace.getEventList().get(0)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); @@ -402,7 +395,7 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { assertEquals(1, trace.getEventList().size()); event = span2.getEvent(); attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.CALLER_SERVICE_NAME, createAttribute(service2)); + attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); assertEquals(event, trace.getEventList().get(0)); } From ecb328ad6de85c4e8ba82cd387c5b4544f9eee35 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Mon, 30 Oct 2023 19:56:31 +0530 Subject: [PATCH 04/13] Addressed review comments --- raw-spans-grouper/helm/values.yaml | 5 -- .../rawspansgrouper/IpIdentityValidator.java | 16 ++++ .../RawSpanGrouperConstants.java | 3 +- .../core/rawspansgrouper/RawSpansGrouper.java | 23 +++-- .../rawspansgrouper/RawSpansProcessor.java | 89 ++++++++----------- .../resources/configs/common/application.conf | 2 + .../raw-spans-grouper/application.conf | 2 +- .../avro/IpResolutionStateStoreValue.avdl | 6 -- .../src/main/avro/SpanMetadata.avdl | 6 ++ ...nExitSpanIdentity.avdl => IpIdentity.avdl} | 4 +- 10 files changed, 79 insertions(+), 77 deletions(-) create mode 100644 raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java delete mode 100644 span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl create mode 100644 span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl rename span-normalizer/span-normalizer-api/src/main/avro/trace-identity/{IpResolutionExitSpanIdentity.avdl => IpIdentity.avdl} (65%) diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 3c1605692..028f87f4b 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -169,11 +169,6 @@ kafka-topic-creator: partitions: 8 configs: cleanup.policy: "[compact, delete]" - raw-spans-to-structured-traces-grouping-job-mirroring-exit-spans-state-store-changelog: - replicationFactor: 3 - partitions: 8 - configs: - cleanup.policy: "[compact, delete]" zookeeper: address: zookeeper:2181 diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java new file mode 100644 index 000000000..c7059118d --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java @@ -0,0 +1,16 @@ +package org.hypertrace.core.rawspansgrouper; + +import java.util.Objects; +import org.hypertrace.core.spannormalizer.IpIdentity; + +public class IpIdentityValidator { + public static boolean isValid(IpIdentity ipIdentity) { + if (Objects.isNull(ipIdentity.getEnvironment()) || Objects.isNull(ipIdentity.getTenantId())) { + return false; + } + + return Objects.nonNull(ipIdentity.getHostAddr()) + || Objects.nonNull(ipIdentity.getPeerAddr()) + || Objects.nonNull(ipIdentity.getPeerPort()); + } +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index 0012098c0..bd9bef361 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -8,12 +8,13 @@ public class RawSpanGrouperConstants { public static final String RAW_SPANS_GROUPER_JOB_CONFIG = "raw-spans-grouper-job-config"; public static final String SPAN_STATE_STORE_NAME = "span-data-store"; public static final String TRACE_STATE_STORE = "trace-state-store"; - public static final String MIRRORING_EXIT_SPANS_STATE_STORE = "mirroring-exit-spans-state-store"; + public static final String IPS_TO_SPAN_METADATA_STATE_STORE = "ips-to-span-metadata-store"; public static final String OUTPUT_TOPIC_PRODUCER = "output-topic-producer"; public static final String SPANS_PER_TRACE_METRIC = "spans_per_trace"; public static final String TRACE_CREATION_TIME = "trace.creation.time"; public static final String DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY = "dataflow.metriccollection.sampling.percent"; + public static final String AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG = "agent.enabled.attribute.name"; public static final String INFLIGHT_TRACE_MAX_SPAN_COUNT = "max.span.count"; public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java index 3ebd1d674..487608295 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java @@ -1,7 +1,7 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.MIRRORING_EXIT_SPANS_STATE_STORE; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.IPS_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; @@ -27,9 +27,9 @@ import org.hypertrace.core.kafkastreams.framework.partitioner.GroupPartitionerBuilder; import org.hypertrace.core.kafkastreams.framework.partitioner.KeyHashPartitioner; import org.hypertrace.core.serviceframework.config.ConfigClient; -import org.hypertrace.core.spannormalizer.IpResolutionExitSpanIdentity; -import org.hypertrace.core.spannormalizer.IpResolutionStateStoreValue; +import org.hypertrace.core.spannormalizer.IpIdentity; import org.hypertrace.core.spannormalizer.SpanIdentity; +import org.hypertrace.core.spannormalizer.SpanMetadata; import org.hypertrace.core.spannormalizer.TraceIdentity; import org.hypertrace.core.spannormalizer.TraceState; import org.slf4j.Logger; @@ -75,17 +75,16 @@ public StreamsBuilder buildTopology( Stores.persistentKeyValueStore(SPAN_STATE_STORE_NAME), keySerde, valueSerde) .withCachingEnabled(); - StoreBuilder> - mirroringExitSpansStoreBuilder = - Stores.keyValueStoreBuilder( - Stores.persistentKeyValueStore(MIRRORING_EXIT_SPANS_STATE_STORE), - keySerde, - valueSerde) - .withCachingEnabled(); + StoreBuilder> ipsToSpanMetadataStateStoreBuilder = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(IPS_TO_SPAN_METADATA_STATE_STORE), + keySerde, + valueSerde) + .withCachingEnabled(); streamsBuilder.addStateStore(spanStoreBuilder); streamsBuilder.addStateStore(traceStateStoreBuilder); - streamsBuilder.addStateStore(mirroringExitSpansStoreBuilder); + streamsBuilder.addStateStore(ipsToSpanMetadataStateStoreBuilder); StreamPartitioner groupPartitioner = new GroupPartitionerBuilder() @@ -106,7 +105,7 @@ public StreamsBuilder buildTopology( Named.as(RawSpansProcessor.class.getSimpleName()), SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, - MIRRORING_EXIT_SPANS_STATE_STORE) + IPS_TO_SPAN_METADATA_STATE_STORE) .to(outputTopic, outputTopicProducer); return streamsBuilder; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index e11c07461..f068212c1 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -1,11 +1,12 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.datamodel.shared.AvroBuilderCache.fastNewBuilder; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DROPPED_SPANS_COUNTER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INFLIGHT_TRACE_MAX_SPAN_COUNT; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.MIRRORING_EXIT_SPANS_STATE_STORE; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.IPS_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; @@ -48,9 +49,9 @@ import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; -import org.hypertrace.core.spannormalizer.IpResolutionExitSpanIdentity; -import org.hypertrace.core.spannormalizer.IpResolutionStateStoreValue; +import org.hypertrace.core.spannormalizer.IpIdentity; import org.hypertrace.core.spannormalizer.SpanIdentity; +import org.hypertrace.core.spannormalizer.SpanMetadata; import org.hypertrace.core.spannormalizer.TraceIdentity; import org.hypertrace.core.spannormalizer.TraceState; import org.hypertrace.semantic.convention.utils.http.HttpSemanticConventionUtils; @@ -71,15 +72,12 @@ public class RawSpansProcessor private static final Logger logger = LoggerFactory.getLogger(RawSpansProcessor.class); private static final String PROCESSING_LATENCY_TIMER = "hypertrace.rawspansgrouper.processing.latency"; - private static final String MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG = - "mirroring.span.attribute.name"; private static final ConcurrentMap tenantToSpansGroupingTimer = new ConcurrentHashMap<>(); private ProcessorContext context; private KeyValueStore spanStore; private KeyValueStore traceStateStore; - private KeyValueStore - mirroringExitSpansStateStore; + private KeyValueStore ipsToSpanMetadataStateStore; private long groupingWindowTimeoutMs; private To outputTopic; private double dataflowSamplingPercent = -1; @@ -93,7 +91,7 @@ public class RawSpansProcessor // counter for number of truncated traces per tenant private static final ConcurrentMap truncatedTracesCounter = new ConcurrentHashMap<>(); - private String mirroringSpanAttributeName; + private String agentEnabledAttributeName; private RawSpansGrouperUtils rawSpansGrouperUtils; @Override @@ -103,13 +101,13 @@ public void init(ProcessorContext context) { (KeyValueStore) context.getStateStore(SPAN_STATE_STORE_NAME); this.traceStateStore = (KeyValueStore) context.getStateStore(TRACE_STATE_STORE); - this.mirroringExitSpansStateStore = - (KeyValueStore) - context.getStateStore(MIRRORING_EXIT_SPANS_STATE_STORE); + this.ipsToSpanMetadataStateStore = + (KeyValueStore) + context.getStateStore(IPS_TO_SPAN_METADATA_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); - this.mirroringSpanAttributeName = - jobConfig.hasPath(MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG) - ? jobConfig.getString(MIRRORING_SPAN_ATTRIBUTE_NAME_CONFIG) + this.agentEnabledAttributeName = + jobConfig.hasPath(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) + ? jobConfig.getString(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) : null; this.groupingWindowTimeoutMs = jobConfig.getLong(SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY) * 1000; @@ -147,7 +145,7 @@ public KeyValue transform(TraceIdentity key, Raw } Event event = value.getEvent(); - if (isMirroringSpan(event)) { + if (isPeerServiceNameIdentificationRequired(event)) { processMirroringSpan(key, value, traceState, currentTimeMs); return null; } @@ -210,52 +208,43 @@ private void processMirroringSpan( boolean firstEntry = (traceState == null); final Optional maybeEnvironment = HttpSemanticConventionUtils.getEnvironmentForSpan(event); - if (SpanSemanticConventionUtils.isClientSpanForOCFormat( event.getAttributes().getAttributeMap())) { final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); final Optional maybePeerPort = HttpSemanticConventionUtils.getPeerPort(event); - if (maybeEnvironment.isPresent() - && maybeHostAddr.isPresent() - && maybePeerAddr.isPresent() - && maybePeerPort.isPresent()) { - final String serviceName = event.getServiceName(); - mirroringExitSpansStateStore.put( - IpResolutionExitSpanIdentity.newBuilder() - .setTenantId(tenantId) - .setEnvironment(maybeEnvironment.get()) - .setHostAddr(maybeHostAddr.get()) - .setPeerAddr(maybePeerAddr.get()) - .setPeerPort(maybePeerPort.get()) - .build(), - IpResolutionStateStoreValue.newBuilder().setServiceName(serviceName).build()); + final String serviceName = event.getServiceName(); + final IpIdentity ipIdentity = + IpIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(maybeEnvironment.orElse(null)) + .setHostAddr(maybeHostAddr.orElse(null)) + .setPeerAddr(maybePeerAddr.orElse(null)) + .setPeerPort(maybePeerPort.orElse(null)) + .build(); + if (IpIdentityValidator.isValid(ipIdentity)) { + ipsToSpanMetadataStateStore.put( + ipIdentity, SpanMetadata.newBuilder().setServiceName(serviceName).build()); } } else { final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); final Optional maybeHostPort = HttpSemanticConventionUtils.getHostPort(event); - if (maybeEnvironment.isPresent() - && maybePeerAddr.isPresent() - && maybeHostAddr.isPresent() - && maybeHostPort.isPresent()) { - final IpResolutionExitSpanIdentity ipResolutionIdentity = - IpResolutionExitSpanIdentity.newBuilder() - .setTenantId(tenantId) - .setEnvironment(maybeEnvironment.get()) - .setHostAddr(maybePeerAddr.get()) - .setPeerAddr(maybeHostAddr.get()) - .setPeerPort(maybeHostPort.get()) - .build(); - final IpResolutionStateStoreValue ipResolutionStateStoreValue = - mirroringExitSpansStateStore.get(ipResolutionIdentity); - if (Objects.nonNull(ipResolutionStateStoreValue)) { + final IpIdentity ipIdentity = + IpIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(maybeEnvironment.orElse(null)) + .setHostAddr(maybePeerAddr.orElse(null)) + .setPeerAddr(maybeHostAddr.orElse(null)) + .setPeerPort(maybeHostPort.orElse(null)) + .build(); + if (IpIdentityValidator.isValid(ipIdentity)) { + final SpanMetadata spanMetadata = ipsToSpanMetadataStateStore.get(ipIdentity); + if (Objects.nonNull(spanMetadata)) { event .getAttributes() .getAttributeMap() - .put( - PEER_SERVICE_NAME, - AttributeValueCreator.create(ipResolutionStateStoreValue.getServiceName())); + .put(PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); } } } @@ -269,7 +258,7 @@ private void processMirroringSpan( context.forward(key, trace, outputTopic); } - private boolean isMirroringSpan(Event event) { + private boolean isPeerServiceNameIdentificationRequired(Event event) { final Attributes attributes = event.getAttributes(); if (Objects.isNull(attributes)) { return false; @@ -282,7 +271,7 @@ private boolean isMirroringSpan(Event event) { final String value = attributeMap - .getOrDefault(this.mirroringSpanAttributeName, AttributeValue.newBuilder().build()) + .getOrDefault(this.agentEnabledAttributeName, AttributeValue.newBuilder().build()) .getValue(); return Objects.nonNull(value) && Boolean.parseBoolean(value); } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf index cf10638b4..d05f3bca6 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf @@ -60,3 +60,5 @@ metrics.reporter { } dataflow.metriccollection.sampling.percent = 10.0 + +agent.enabled.attribute.name = "mirroring.enabled" diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index 04947553e..0dc98178e 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -45,4 +45,4 @@ span.groupby.session.window.graceperiod.ms = 100 dataflow.metriccollection.sampling.percent = 10.0 -mirroring.span.attribute.name = "mirroring.enabled" +agent.enabled.attribute.name = "mirroring.enabled" diff --git a/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl b/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl deleted file mode 100644 index 76644a2eb..000000000 --- a/span-normalizer/span-normalizer-api/src/main/avro/IpResolutionStateStoreValue.avdl +++ /dev/null @@ -1,6 +0,0 @@ -@namespace("org.hypertrace.core.spannormalizer") -protocol IpResolutionStateStoreValueProtocol { - record IpResolutionStateStoreValue { - string service_name; - } -} diff --git a/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl new file mode 100644 index 000000000..0961391db --- /dev/null +++ b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl @@ -0,0 +1,6 @@ +@namespace("org.hypertrace.core.spannormalizer") +protocol SpanMetadataProtocol { + record SpanMetadata { + string service_name; + } +} diff --git a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl similarity index 65% rename from span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl rename to span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl index 5b147aa31..f963d78ad 100644 --- a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpResolutionExitSpanIdentity.avdl +++ b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl @@ -1,6 +1,6 @@ @namespace("org.hypertrace.core.spannormalizer") -protocol IpResolutionExitSpanIdentityProtocol { - record IpResolutionExitSpanIdentity { +protocol IpIdentityProtocol { + record IpIdentity { string tenant_id; string environment; string host_addr; From 4680b9de19e5a039dcc267c868c083da6b070206 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Wed, 10 Jan 2024 18:47:21 +0530 Subject: [PATCH 05/13] comment --- .../rawspansgrouper/RawSpansProcessor.java | 18 +- .../rawspansgrouper/RawSpansGrouperTest.java | 318 +++++++++--------- 2 files changed, 172 insertions(+), 164 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 29caa3297..e47f23fee 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -48,7 +48,6 @@ import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.Timestamps; import org.hypertrace.core.datamodel.shared.HexUtils; -import org.hypertrace.core.datamodel.shared.trace.AttributeValueCreator; import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; @@ -269,10 +268,19 @@ private void processSpanForPeerServiceNameIdentification( if (IpIdentityValidator.isValid(ipIdentity)) { final SpanMetadata spanMetadata = ipsToSpanMetadataStateStore.get(ipIdentity); if (Objects.nonNull(spanMetadata)) { - event - .getAttributes() - .getAttributeMap() - .put(PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); + // Instead of updating the span, adding a debug log to print the service correlation + logger.debug( + "Adding {} as: {} in spanId: {} with service name: {}", + PEER_SERVICE_NAME, + spanMetadata.getServiceName(), + HexUtils.getHex(event.getEventId()), + event.getServiceName()); + + // event + // .getAttributes() + // .getAttributeMap() + // .put(PEER_SERVICE_NAME, + // AttributeValueCreator.create(spanMetadata.getServiceName())); } } } diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index 0a2149786..bf65a6728 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.test.TestRecord; import org.hypertrace.core.datamodel.AttributeValue; -import org.hypertrace.core.datamodel.Attributes; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -305,129 +304,130 @@ private long advanceAndSyncClockMock(long messageTime, Clock clock, long advance return finalMessageTime; } - @Test - @SetEnvironmentVariable(key = "SERVICE_NAME", value = "raw-spans-grouper") - void testMirroringSpansGrouping(@TempDir Path tempDir) { - File file = tempDir.resolve("state").toFile(); - - RawSpansGrouper underTest = new RawSpansGrouper(ConfigClientFactory.getClient()); - Config config = - ConfigFactory.parseURL( - getClass().getClassLoader().getResource("configs/raw-spans-grouper/application.conf")); - - Map baseProps = underTest.getBaseStreamsConfig(); - Map streamsProps = underTest.getStreamsConfig(config); - baseProps.forEach(streamsProps::put); - Map mergedProps = streamsProps; - - mergedProps.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); - mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); - mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); - - StreamsBuilder streamsBuilder = - underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); - - Properties props = new Properties(); - mergedProps.forEach(props::put); - - Serde defaultValueSerde = new StreamsConfig(mergedProps).defaultValueSerde(); - - Serde traceIdentitySerde = new StreamsConfig(mergedProps).defaultKeySerde(); - - TopologyTestDriver td = new TopologyTestDriver(streamsBuilder.build(), props); - TestInputTopic inputTopic = - td.createInputTopic( - config.getString(RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY), - traceIdentitySerde.serializer(), - defaultValueSerde.serializer()); - - TestOutputTopic outputTopic = - td.createOutputTopic( - config.getString(RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY), - traceIdentitySerde.deserializer(), - defaultValueSerde.deserializer()); - - String hostAddr1 = "1.2.3.4"; - String hostAddr2 = "1.2.3.5"; - String hostAddr3 = "1.2.3.6"; - String hostPort2 = "5000"; - String hostPort3 = "6000"; - String service1 = "service1"; - String service2 = "service2"; - String service3 = "service3"; - - String tenantId = "tenant1"; - RawSpan span1 = - RawSpan.newBuilder() - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .setCustomerId(tenantId) - .setEvent( - createMirroringEvent( - "event-1", tenantId, service1, "client", hostAddr1, "1234", hostAddr2, - hostPort2)) - .build(); - RawSpan span2 = - RawSpan.newBuilder() - .setTraceId(ByteBuffer.wrap("trace-2".getBytes())) - .setCustomerId(tenantId) - .setEvent( - createMirroringEvent( - "event-2", tenantId, service3, "server", hostAddr3, hostPort3, hostAddr2, - "5678")) - .build(); - RawSpan span3 = - RawSpan.newBuilder() - .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) - .setCustomerId(tenantId) - .setEvent( - createMirroringEvent( - "event-3", tenantId, service2, "server", hostAddr2, hostPort2, hostAddr1, - "1234")) - .build(); - RawSpan span4 = - RawSpan.newBuilder() - .setTraceId(ByteBuffer.wrap("trace-4".getBytes())) - .setCustomerId(tenantId) - .setEvent( - createMirroringEvent( - "event-4", tenantId, service2, "client", hostAddr2, "5678", hostAddr3, - hostPort3)) - .build(); - - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); - StructuredTrace trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - assertEquals(span1.getEvent(), trace.getEventList().get(0)); - - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); - trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - assertEquals(span2.getEvent(), trace.getEventList().get(0)); - - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span3); - trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - Event event = span3.getEvent(); - Map attributeMap = - new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); - event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); - assertEquals(event, trace.getEventList().get(0)); - - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); - trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - assertEquals(span4.getEvent(), trace.getEventList().get(0)); - - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); - trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - event = span2.getEvent(); - attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); - event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); - assertEquals(event, trace.getEventList().get(0)); - } + // @Test + // @SetEnvironmentVariable(key = "SERVICE_NAME", value = "raw-spans-grouper") + // void testMirroringSpansGrouping(@TempDir Path tempDir) { + // File file = tempDir.resolve("state").toFile(); + // + // RawSpansGrouper underTest = new RawSpansGrouper(ConfigClientFactory.getClient()); + // Config config = + // ConfigFactory.parseURL( + // + // getClass().getClassLoader().getResource("configs/raw-spans-grouper/application.conf")); + // + // Map baseProps = underTest.getBaseStreamsConfig(); + // Map streamsProps = underTest.getStreamsConfig(config); + // baseProps.forEach(streamsProps::put); + // Map mergedProps = streamsProps; + // + // mergedProps.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); + // mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); + // mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); + // + // StreamsBuilder streamsBuilder = + // underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); + // + // Properties props = new Properties(); + // mergedProps.forEach(props::put); + // + // Serde defaultValueSerde = new StreamsConfig(mergedProps).defaultValueSerde(); + // + // Serde traceIdentitySerde = new StreamsConfig(mergedProps).defaultKeySerde(); + // + // TopologyTestDriver td = new TopologyTestDriver(streamsBuilder.build(), props); + // TestInputTopic inputTopic = + // td.createInputTopic( + // config.getString(RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY), + // traceIdentitySerde.serializer(), + // defaultValueSerde.serializer()); + // + // TestOutputTopic outputTopic = + // td.createOutputTopic( + // config.getString(RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY), + // traceIdentitySerde.deserializer(), + // defaultValueSerde.deserializer()); + // + // String hostAddr1 = "1.2.3.4"; + // String hostAddr2 = "1.2.3.5"; + // String hostAddr3 = "1.2.3.6"; + // String hostPort2 = "5000"; + // String hostPort3 = "6000"; + // String service1 = "service1"; + // String service2 = "service2"; + // String service3 = "service3"; + // + // String tenantId = "tenant1"; + // RawSpan span1 = + // RawSpan.newBuilder() + // .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + // .setCustomerId(tenantId) + // .setEvent( + // createMirroringEvent( + // "event-1", tenantId, service1, "client", hostAddr1, "1234", hostAddr2, + // hostPort2)) + // .build(); + // RawSpan span2 = + // RawSpan.newBuilder() + // .setTraceId(ByteBuffer.wrap("trace-2".getBytes())) + // .setCustomerId(tenantId) + // .setEvent( + // createMirroringEvent( + // "event-2", tenantId, service3, "server", hostAddr3, hostPort3, hostAddr2, + // "5678")) + // .build(); + // RawSpan span3 = + // RawSpan.newBuilder() + // .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) + // .setCustomerId(tenantId) + // .setEvent( + // createMirroringEvent( + // "event-3", tenantId, service2, "server", hostAddr2, hostPort2, hostAddr1, + // "1234")) + // .build(); + // RawSpan span4 = + // RawSpan.newBuilder() + // .setTraceId(ByteBuffer.wrap("trace-4".getBytes())) + // .setCustomerId(tenantId) + // .setEvent( + // createMirroringEvent( + // "event-4", tenantId, service2, "client", hostAddr2, "5678", hostAddr3, + // hostPort3)) + // .build(); + // + // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); + // StructuredTrace trace = outputTopic.readValue(); + // assertEquals(1, trace.getEventList().size()); + // assertEquals(span1.getEvent(), trace.getEventList().get(0)); + // + // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + // trace = outputTopic.readValue(); + // assertEquals(1, trace.getEventList().size()); + // assertEquals(span2.getEvent(), trace.getEventList().get(0)); + // + // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span3); + // trace = outputTopic.readValue(); + // assertEquals(1, trace.getEventList().size()); + // Event event = span3.getEvent(); + // Map attributeMap = + // new HashMap<>(event.getAttributes().getAttributeMap()); + // attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); + // event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + // assertEquals(event, trace.getEventList().get(0)); + // + // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); + // trace = outputTopic.readValue(); + // assertEquals(1, trace.getEventList().size()); + // assertEquals(span4.getEvent(), trace.getEventList().get(0)); + // + // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + // trace = outputTopic.readValue(); + // assertEquals(1, trace.getEventList().size()); + // event = span2.getEvent(); + // attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); + // attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); + // event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + // assertEquals(event, trace.getEventList().get(0)); + // } private Event createEvent(String eventId, String tenantId) { return Event.newBuilder() @@ -437,41 +437,41 @@ private Event createEvent(String eventId, String tenantId) { .build(); } - private Event createMirroringEvent( - String eventId, - String tenantId, - String service, - String spanKind, - String hostAddr, - String hostPort, - String peerAddr, - String peerPort) { - return Event.newBuilder() - .setCustomerId(tenantId) - .setServiceName(service) - .setEventId(ByteBuffer.wrap(eventId.getBytes())) - .setStartTimeMillis(System.currentTimeMillis()) - .setAttributes( - Attributes.newBuilder() - .setAttributeMap( - Map.of( - "mirroring.enabled", - createAttribute("true"), - "deployment.environment", - createAttribute("environment"), - "span.kind", - createAttribute(spanKind), - "net.sock.host.addr", - createAttribute(hostAddr), - "net.sock.host.port", - createAttribute(hostPort), - "net.sock.peer.addr", - createAttribute(peerAddr), - "net.sock.peer.port", - createAttribute(peerPort))) - .build()) - .build(); - } + // private Event createMirroringEvent( + // String eventId, + // String tenantId, + // String service, + // String spanKind, + // String hostAddr, + // String hostPort, + // String peerAddr, + // String peerPort) { + // return Event.newBuilder() + // .setCustomerId(tenantId) + // .setServiceName(service) + // .setEventId(ByteBuffer.wrap(eventId.getBytes())) + // .setStartTimeMillis(System.currentTimeMillis()) + // .setAttributes( + // Attributes.newBuilder() + // .setAttributeMap( + // Map.of( + // "mirroring.enabled", + // createAttribute("true"), + // "deployment.environment", + // createAttribute("environment"), + // "span.kind", + // createAttribute(spanKind), + // "net.sock.host.addr", + // createAttribute(hostAddr), + // "net.sock.host.port", + // createAttribute(hostPort), + // "net.sock.peer.addr", + // createAttribute(peerAddr), + // "net.sock.peer.port", + // createAttribute(peerPort))) + // .build()) + // .build(); + // } private AttributeValue createAttribute(String value) { return AttributeValue.newBuilder().setValue(value).build(); From 91d57ec6de35e2953c9f5920048ab9e35702185e Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Tue, 16 Jan 2024 10:27:25 +0530 Subject: [PATCH 06/13] add flags --- .../templates/raw-spans-grouper-config.yaml | 8 + raw-spans-grouper/helm/values.yaml | 2 + .../RawSpanGrouperConstants.java | 4 + .../rawspansgrouper/RawSpansProcessor.java | 33 +- .../resources/configs/common/application.conf | 4 + .../rawspansgrouper/RawSpansGrouperTest.java | 318 +++++++++--------- .../raw-spans-grouper/application.conf | 4 + .../templates/span-normalizer-config.yaml | 4 + .../rawspan/ByPassPredicate.java | 16 +- 9 files changed, 227 insertions(+), 166 deletions(-) diff --git a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml index ca65a46e4..fa5530f6a 100644 --- a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml +++ b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml @@ -85,3 +85,11 @@ data: {{- if hasKey .Values.rawSpansGrouperConfig "traceEmitPunctuatorFrequency" }} trace.emit.punctuator.frequency = {{ .Values.rawSpansGrouperConfig.traceEmitPunctuatorFrequency }} {{- end }} + + {{- if hasKey .Values.rawSpansGrouperConfig "serviceCorrelationEnabledCustomers" }} + service.correlation.enabled.customers = {{ .Values.rawSpansGrouperConfig.serviceCorrelationEnabledCustomers | toJson }} + {{- end }} + + {{- if hasKey .Values.rawSpansGrouperConfig "peerServiceNameSpansEnabledCustomers" }} + peer.service.name.spans.enabled.customers = {{ .Values.rawSpansGrouperConfig.peerServiceNameSpansEnabledCustomers | toJson }} + {{- end }} diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 139c0855c..231869d1a 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -133,6 +133,8 @@ rawSpansGrouperConfig: groupPartitionerConfigServiceHost: config-service groupPartitionerConfigServicePort: 50104 traceEmitPunctuatorFrequency: 5s + serviceCorrelationEnabledCustomers: [] + peerServiceNameSpansEnabledCustomers: [] span: groupby: diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index 1c86eecdf..1a60f40d8 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -15,6 +15,10 @@ public class RawSpanGrouperConstants { public static final String DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY = "dataflow.metriccollection.sampling.percent"; public static final String AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG = "agent.enabled.attribute.name"; + public static final String SERVICE_CORRELATION_ENABLED_CUSTOMERS = + "service.correlation.enabled.customers"; + public static final String PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS = + "peer.service.name.spans.enabled.customers"; public static final String INFLIGHT_TRACE_MAX_SPAN_COUNT = "max.span.count"; public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index e47f23fee..a7c2fc901 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -10,7 +10,9 @@ import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.IPS_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SERVICE_CORRELATION_ENABLED_CUSTOMERS; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_STATE_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR; @@ -26,6 +28,7 @@ import java.time.Clock; import java.time.Duration; import java.time.Instant; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,6 +51,7 @@ import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.Timestamps; import org.hypertrace.core.datamodel.shared.HexUtils; +import org.hypertrace.core.datamodel.shared.trace.AttributeValueCreator; import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; @@ -75,6 +79,7 @@ public class RawSpansProcessor private static final Logger logger = LoggerFactory.getLogger(RawSpansProcessor.class); private static final String PROCESSING_LATENCY_TIMER = "hypertrace.rawspansgrouper.processing.latency"; + private static final String ALL = "*"; private static final ConcurrentMap tenantToSpansGroupingTimer = new ConcurrentHashMap<>(); // counter for number of spans dropped per tenant @@ -96,6 +101,8 @@ public class RawSpansProcessor private TraceEmitPunctuator traceEmitPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; private String agentEnabledAttributeName; + private List serviceCorrelationEnabledCustomers; + private List peerServiceNameInSpansEnabledTenants; private RawSpansGrouperUtils rawSpansGrouperUtils; public RawSpansProcessor(Clock clock) { @@ -113,6 +120,14 @@ public void init(ProcessorContext context) { jobConfig.hasPath(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) ? jobConfig.getString(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) : null; + this.serviceCorrelationEnabledCustomers = + jobConfig.hasPath(SERVICE_CORRELATION_ENABLED_CUSTOMERS) + ? jobConfig.getStringList(SERVICE_CORRELATION_ENABLED_CUSTOMERS) + : Collections.emptyList(); + this.peerServiceNameInSpansEnabledTenants = + jobConfig.hasPath(PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS) + ? jobConfig.getStringList(PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS) + : Collections.emptyList(); this.groupingWindowTimeoutMs = jobConfig.getLong(SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY) * 1000; @@ -276,11 +291,14 @@ private void processSpanForPeerServiceNameIdentification( HexUtils.getHex(event.getEventId()), event.getServiceName()); - // event - // .getAttributes() - // .getAttributeMap() - // .put(PEER_SERVICE_NAME, - // AttributeValueCreator.create(spanMetadata.getServiceName())); + if (peerServiceNameInSpansEnabledTenants.contains(tenantId) + || peerServiceNameInSpansEnabledTenants.contains(ALL)) { + event + .getAttributes() + .getAttributeMap() + .put( + PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); + } } } } @@ -309,7 +327,10 @@ private boolean isPeerServiceNameIdentificationRequired(Event event) { attributeMap .getOrDefault(this.agentEnabledAttributeName, AttributeValue.newBuilder().build()) .getValue(); - return Objects.nonNull(value) && Boolean.parseBoolean(value); + return Objects.nonNull(value) + && Boolean.parseBoolean(value) + && (this.serviceCorrelationEnabledCustomers.contains(event.getCustomerId()) + || this.serviceCorrelationEnabledCustomers.contains(ALL)); } private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf index 08d8e9626..893c6bcf4 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf @@ -68,3 +68,7 @@ trace.emit.punctuator { } agent.enabled.attribute.name = "mirroring.enabled" + +service.correlation.enabled.customers = [] + +peer.service.name.spans.enabled.customers = [] diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index bf65a6728..0a2149786 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.streams.TopologyTestDriver; import org.apache.kafka.streams.test.TestRecord; import org.hypertrace.core.datamodel.AttributeValue; +import org.hypertrace.core.datamodel.Attributes; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -304,130 +305,129 @@ private long advanceAndSyncClockMock(long messageTime, Clock clock, long advance return finalMessageTime; } - // @Test - // @SetEnvironmentVariable(key = "SERVICE_NAME", value = "raw-spans-grouper") - // void testMirroringSpansGrouping(@TempDir Path tempDir) { - // File file = tempDir.resolve("state").toFile(); - // - // RawSpansGrouper underTest = new RawSpansGrouper(ConfigClientFactory.getClient()); - // Config config = - // ConfigFactory.parseURL( - // - // getClass().getClassLoader().getResource("configs/raw-spans-grouper/application.conf")); - // - // Map baseProps = underTest.getBaseStreamsConfig(); - // Map streamsProps = underTest.getStreamsConfig(config); - // baseProps.forEach(streamsProps::put); - // Map mergedProps = streamsProps; - // - // mergedProps.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); - // mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); - // mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); - // - // StreamsBuilder streamsBuilder = - // underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); - // - // Properties props = new Properties(); - // mergedProps.forEach(props::put); - // - // Serde defaultValueSerde = new StreamsConfig(mergedProps).defaultValueSerde(); - // - // Serde traceIdentitySerde = new StreamsConfig(mergedProps).defaultKeySerde(); - // - // TopologyTestDriver td = new TopologyTestDriver(streamsBuilder.build(), props); - // TestInputTopic inputTopic = - // td.createInputTopic( - // config.getString(RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY), - // traceIdentitySerde.serializer(), - // defaultValueSerde.serializer()); - // - // TestOutputTopic outputTopic = - // td.createOutputTopic( - // config.getString(RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY), - // traceIdentitySerde.deserializer(), - // defaultValueSerde.deserializer()); - // - // String hostAddr1 = "1.2.3.4"; - // String hostAddr2 = "1.2.3.5"; - // String hostAddr3 = "1.2.3.6"; - // String hostPort2 = "5000"; - // String hostPort3 = "6000"; - // String service1 = "service1"; - // String service2 = "service2"; - // String service3 = "service3"; - // - // String tenantId = "tenant1"; - // RawSpan span1 = - // RawSpan.newBuilder() - // .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - // .setCustomerId(tenantId) - // .setEvent( - // createMirroringEvent( - // "event-1", tenantId, service1, "client", hostAddr1, "1234", hostAddr2, - // hostPort2)) - // .build(); - // RawSpan span2 = - // RawSpan.newBuilder() - // .setTraceId(ByteBuffer.wrap("trace-2".getBytes())) - // .setCustomerId(tenantId) - // .setEvent( - // createMirroringEvent( - // "event-2", tenantId, service3, "server", hostAddr3, hostPort3, hostAddr2, - // "5678")) - // .build(); - // RawSpan span3 = - // RawSpan.newBuilder() - // .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) - // .setCustomerId(tenantId) - // .setEvent( - // createMirroringEvent( - // "event-3", tenantId, service2, "server", hostAddr2, hostPort2, hostAddr1, - // "1234")) - // .build(); - // RawSpan span4 = - // RawSpan.newBuilder() - // .setTraceId(ByteBuffer.wrap("trace-4".getBytes())) - // .setCustomerId(tenantId) - // .setEvent( - // createMirroringEvent( - // "event-4", tenantId, service2, "client", hostAddr2, "5678", hostAddr3, - // hostPort3)) - // .build(); - // - // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); - // StructuredTrace trace = outputTopic.readValue(); - // assertEquals(1, trace.getEventList().size()); - // assertEquals(span1.getEvent(), trace.getEventList().get(0)); - // - // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); - // trace = outputTopic.readValue(); - // assertEquals(1, trace.getEventList().size()); - // assertEquals(span2.getEvent(), trace.getEventList().get(0)); - // - // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span3); - // trace = outputTopic.readValue(); - // assertEquals(1, trace.getEventList().size()); - // Event event = span3.getEvent(); - // Map attributeMap = - // new HashMap<>(event.getAttributes().getAttributeMap()); - // attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); - // event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); - // assertEquals(event, trace.getEventList().get(0)); - // - // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); - // trace = outputTopic.readValue(); - // assertEquals(1, trace.getEventList().size()); - // assertEquals(span4.getEvent(), trace.getEventList().get(0)); - // - // inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); - // trace = outputTopic.readValue(); - // assertEquals(1, trace.getEventList().size()); - // event = span2.getEvent(); - // attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - // attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); - // event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); - // assertEquals(event, trace.getEventList().get(0)); - // } + @Test + @SetEnvironmentVariable(key = "SERVICE_NAME", value = "raw-spans-grouper") + void testMirroringSpansGrouping(@TempDir Path tempDir) { + File file = tempDir.resolve("state").toFile(); + + RawSpansGrouper underTest = new RawSpansGrouper(ConfigClientFactory.getClient()); + Config config = + ConfigFactory.parseURL( + getClass().getClassLoader().getResource("configs/raw-spans-grouper/application.conf")); + + Map baseProps = underTest.getBaseStreamsConfig(); + Map streamsProps = underTest.getStreamsConfig(config); + baseProps.forEach(streamsProps::put); + Map mergedProps = streamsProps; + + mergedProps.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, SpecificAvroSerde.class); + mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); + mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); + + StreamsBuilder streamsBuilder = + underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); + + Properties props = new Properties(); + mergedProps.forEach(props::put); + + Serde defaultValueSerde = new StreamsConfig(mergedProps).defaultValueSerde(); + + Serde traceIdentitySerde = new StreamsConfig(mergedProps).defaultKeySerde(); + + TopologyTestDriver td = new TopologyTestDriver(streamsBuilder.build(), props); + TestInputTopic inputTopic = + td.createInputTopic( + config.getString(RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY), + traceIdentitySerde.serializer(), + defaultValueSerde.serializer()); + + TestOutputTopic outputTopic = + td.createOutputTopic( + config.getString(RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY), + traceIdentitySerde.deserializer(), + defaultValueSerde.deserializer()); + + String hostAddr1 = "1.2.3.4"; + String hostAddr2 = "1.2.3.5"; + String hostAddr3 = "1.2.3.6"; + String hostPort2 = "5000"; + String hostPort3 = "6000"; + String service1 = "service1"; + String service2 = "service2"; + String service3 = "service3"; + + String tenantId = "tenant1"; + RawSpan span1 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-1", tenantId, service1, "client", hostAddr1, "1234", hostAddr2, + hostPort2)) + .build(); + RawSpan span2 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-2".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-2", tenantId, service3, "server", hostAddr3, hostPort3, hostAddr2, + "5678")) + .build(); + RawSpan span3 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-3", tenantId, service2, "server", hostAddr2, hostPort2, hostAddr1, + "1234")) + .build(); + RawSpan span4 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-4".getBytes())) + .setCustomerId(tenantId) + .setEvent( + createMirroringEvent( + "event-4", tenantId, service2, "client", hostAddr2, "5678", hostAddr3, + hostPort3)) + .build(); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); + StructuredTrace trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span1.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span2.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span3); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + Event event = span3.getEvent(); + Map attributeMap = + new HashMap<>(event.getAttributes().getAttributeMap()); + attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); + event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + assertEquals(event, trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals(span4.getEvent(), trace.getEventList().get(0)); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span2); + trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + event = span2.getEvent(); + attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); + attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); + event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + assertEquals(event, trace.getEventList().get(0)); + } private Event createEvent(String eventId, String tenantId) { return Event.newBuilder() @@ -437,41 +437,41 @@ private Event createEvent(String eventId, String tenantId) { .build(); } - // private Event createMirroringEvent( - // String eventId, - // String tenantId, - // String service, - // String spanKind, - // String hostAddr, - // String hostPort, - // String peerAddr, - // String peerPort) { - // return Event.newBuilder() - // .setCustomerId(tenantId) - // .setServiceName(service) - // .setEventId(ByteBuffer.wrap(eventId.getBytes())) - // .setStartTimeMillis(System.currentTimeMillis()) - // .setAttributes( - // Attributes.newBuilder() - // .setAttributeMap( - // Map.of( - // "mirroring.enabled", - // createAttribute("true"), - // "deployment.environment", - // createAttribute("environment"), - // "span.kind", - // createAttribute(spanKind), - // "net.sock.host.addr", - // createAttribute(hostAddr), - // "net.sock.host.port", - // createAttribute(hostPort), - // "net.sock.peer.addr", - // createAttribute(peerAddr), - // "net.sock.peer.port", - // createAttribute(peerPort))) - // .build()) - // .build(); - // } + private Event createMirroringEvent( + String eventId, + String tenantId, + String service, + String spanKind, + String hostAddr, + String hostPort, + String peerAddr, + String peerPort) { + return Event.newBuilder() + .setCustomerId(tenantId) + .setServiceName(service) + .setEventId(ByteBuffer.wrap(eventId.getBytes())) + .setStartTimeMillis(System.currentTimeMillis()) + .setAttributes( + Attributes.newBuilder() + .setAttributeMap( + Map.of( + "mirroring.enabled", + createAttribute("true"), + "deployment.environment", + createAttribute("environment"), + "span.kind", + createAttribute(spanKind), + "net.sock.host.addr", + createAttribute(hostAddr), + "net.sock.host.port", + createAttribute(hostPort), + "net.sock.peer.addr", + createAttribute(peerAddr), + "net.sock.peer.port", + createAttribute(peerPort))) + .build()) + .build(); + } private AttributeValue createAttribute(String value) { return AttributeValue.newBuilder().setValue(value).build(); diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index ae5bd9452..ece8c22e1 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -50,3 +50,7 @@ dataflow.metriccollection.sampling.percent = 10.0 trace.emit.punctuator.frequency = 15s agent.enabled.attribute.name = "mirroring.enabled" + +service.correlation.enabled.customers = ["*"] + +peer.service.name.spans.enabled.customers = ["*"] \ No newline at end of file diff --git a/span-normalizer/helm/templates/span-normalizer-config.yaml b/span-normalizer/helm/templates/span-normalizer-config.yaml index 55c3f830e..0a672ec66 100644 --- a/span-normalizer/helm/templates/span-normalizer-config.yaml +++ b/span-normalizer/helm/templates/span-normalizer-config.yaml @@ -70,6 +70,10 @@ data: {{- if hasKey .Values.spanNormalizerConfig.processor "bypassKey" }} bypass.key = "{{ .Values.spanNormalizerConfig.processor.bypassKey }}" {{- end }} + + {{- if hasKey .Values.spanNormalizerConfig.processor "bypassOverrideTenants" }} + bypass.override.tenants = "{{ .Values.spanNormalizerConfig.processor.bypassOverrideTenants | toJson }}" + {{- end }} {{- if hasKey .Values.spanNormalizerConfig.processor "lateArrivalThresholdDuration" }} late.arrival.threshold.duration = "{{ .Values.spanNormalizerConfig.processor.lateArrivalThresholdDuration }}" diff --git a/span-normalizer/span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/rawspan/ByPassPredicate.java b/span-normalizer/span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/rawspan/ByPassPredicate.java index 3f74060e6..c8a1619a7 100644 --- a/span-normalizer/span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/rawspan/ByPassPredicate.java +++ b/span-normalizer/span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/rawspan/ByPassPredicate.java @@ -1,22 +1,36 @@ package org.hypertrace.core.spannormalizer.rawspan; import com.typesafe.config.Config; +import java.util.Collections; +import java.util.List; import org.apache.kafka.streams.kstream.Predicate; import org.hypertrace.core.datamodel.AttributeValue; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.spannormalizer.TraceIdentity; public class ByPassPredicate implements Predicate { + private static final String ALL = "*"; private static final String SPAN_BYPASSED_CONFIG = "processor.bypass.key"; - private String bypassKey; + private static final String SPAN_BYPASSED_OVERRIDE_CONFIG = "processor.bypass.override.tenants"; + private final String bypassKey; + private final List bypassOverrideTenants; public ByPassPredicate(Config jobConfig) { bypassKey = jobConfig.hasPath(SPAN_BYPASSED_CONFIG) ? jobConfig.getString(SPAN_BYPASSED_CONFIG) : null; + bypassOverrideTenants = + jobConfig.hasPath(SPAN_BYPASSED_OVERRIDE_CONFIG) + ? jobConfig.getStringList(SPAN_BYPASSED_OVERRIDE_CONFIG) + : Collections.emptyList(); } @Override public boolean test(TraceIdentity traceIdentity, RawSpan rawSpan) { + // tenant level spans bypass override + if (bypassOverrideTenants.contains(rawSpan.getCustomerId()) + || bypassOverrideTenants.contains(ALL)) { + return false; + } AttributeValue defaultAttributeValue = AttributeValue.newBuilder().setValue("false").build(); AttributeValue attributeValue = bypassKey != null From 6fadfafd5550c4980983eda078db393c6e175c22 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Tue, 16 Jan 2024 18:07:34 +0530 Subject: [PATCH 07/13] Addressed review comments --- .../templates/raw-spans-grouper-config.yaml | 8 +- raw-spans-grouper/helm/values.yaml | 2 +- .../raw-spans-grouper/build.gradle.kts | 1 + .../PeerIdentityValidator.java | 10 ++ .../RawSpanGrouperConstants.java | 9 +- .../core/rawspansgrouper/RawSpansGrouper.java | 21 +-- .../rawspansgrouper/RawSpansProcessor.java | 161 ++++++++++-------- .../rawspansgrouper/TraceEmitPunctuator.java | 8 +- ...ouperUtils.java => TraceLatencyMeter.java} | 4 +- .../resources/configs/common/application.conf | 5 +- .../rawspansgrouper/RawSpansGrouperTest.java | 9 +- ...lsTest.java => TraceLatencyMeterTest.java} | 10 +- .../raw-spans-grouper/application.conf | 6 +- .../main/avro/trace-identity/IpIdentity.avdl | 10 -- .../avro/trace-identity/PeerIdentity.avdl | 15 ++ 15 files changed, 154 insertions(+), 125 deletions(-) create mode 100644 raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/{RawSpansGrouperUtils.java => TraceLatencyMeter.java} (93%) rename raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/{RawSpansGrouperUtilsTest.java => TraceLatencyMeterTest.java} (70%) delete mode 100644 span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl create mode 100644 span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl diff --git a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml index fa5530f6a..e2ce9a41c 100644 --- a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml +++ b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml @@ -90,6 +90,10 @@ data: service.correlation.enabled.customers = {{ .Values.rawSpansGrouperConfig.serviceCorrelationEnabledCustomers | toJson }} {{- end }} - {{- if hasKey .Values.rawSpansGrouperConfig "peerServiceNameSpansEnabledCustomers" }} - peer.service.name.spans.enabled.customers = {{ .Values.rawSpansGrouperConfig.peerServiceNameSpansEnabledCustomers | toJson }} + {{- if hasKey .Values.rawSpansGrouperConfig "peerCorrelationEnabledAgents" }} + peer.correlation.enabled.agents = {{ .Values.rawSpansGrouperConfig.peerCorrelationEnabledAgents | toJson }} + {{- end }} + + {{- if hasKey .Values.rawSpansGrouperConfig "agentTypeAttributeName" }} + agent.type.attribute.name = {{ .Values.rawSpansGrouperConfig.agentTypeAttributeName }} {{- end }} diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 231869d1a..ed0bea666 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -134,7 +134,7 @@ rawSpansGrouperConfig: groupPartitionerConfigServicePort: 50104 traceEmitPunctuatorFrequency: 5s serviceCorrelationEnabledCustomers: [] - peerServiceNameSpansEnabledCustomers: [] + peerCorrelationEnabledAgents: [] span: groupby: diff --git a/raw-spans-grouper/raw-spans-grouper/build.gradle.kts b/raw-spans-grouper/raw-spans-grouper/build.gradle.kts index eb90fec7e..de03c6d8c 100644 --- a/raw-spans-grouper/raw-spans-grouper/build.gradle.kts +++ b/raw-spans-grouper/raw-spans-grouper/build.gradle.kts @@ -34,6 +34,7 @@ dependencies { } implementation(project(":span-normalizer:span-normalizer-api")) implementation(project(":semantic-convention-utils")) + implementation(project(":hypertrace-trace-enricher:enriched-span-constants")) implementation(libs.hypertrace.data.model) implementation(libs.hypertrace.serviceFramework.framework) implementation(libs.hypertrace.serviceFramework.metrics) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java new file mode 100644 index 000000000..3941967ae --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java @@ -0,0 +1,10 @@ +package org.hypertrace.core.rawspansgrouper; + +import org.hypertrace.core.spannormalizer.PeerIdentity; + +public class PeerIdentityValidator { + + public static boolean isValid(PeerIdentity peerIdentity) { + return IpIdentityValidator.isValid(peerIdentity.getIpIdentity()); + } +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index 1a60f40d8..780992543 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -8,17 +8,17 @@ public class RawSpanGrouperConstants { public static final String RAW_SPANS_GROUPER_JOB_CONFIG = "raw-spans-grouper-job-config"; public static final String SPAN_STATE_STORE_NAME = "span-data-store"; public static final String TRACE_STATE_STORE = "trace-state-store"; - public static final String IPS_TO_SPAN_METADATA_STATE_STORE = "ips-to-span-metadata-store"; + public static final String PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE = + "peer-identity-to-span-metadata-store"; public static final String OUTPUT_TOPIC_PRODUCER = "output-topic-producer"; public static final String SPANS_PER_TRACE_METRIC = "spans_per_trace"; public static final String TRACE_CREATION_TIME = "trace.creation.time"; public static final String DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY = "dataflow.metriccollection.sampling.percent"; - public static final String AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG = "agent.enabled.attribute.name"; + public static final String AGENT_TYPE_ATTRIBUTE_NAME_CONFIG = "agent.type.attribute.name"; public static final String SERVICE_CORRELATION_ENABLED_CUSTOMERS = "service.correlation.enabled.customers"; - public static final String PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS = - "peer.service.name.spans.enabled.customers"; + public static final String PEER_CORRELATION_ENABLED_AGENTS = "peer.correlation.enabled.agents"; public static final String INFLIGHT_TRACE_MAX_SPAN_COUNT = "max.span.count"; public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; @@ -27,5 +27,4 @@ public class RawSpanGrouperConstants { static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = "emitter-store"; static final String TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY = "trace.emit.punctuator.frequency"; - public static final String PEER_SERVICE_NAME = "peer.service.name"; } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java index 622a85ee4..05adcee51 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouper.java @@ -1,9 +1,9 @@ package org.hypertrace.core.rawspansgrouper; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INPUT_TOPIC_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.IPS_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_STATE_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR_STORE_NAME; @@ -32,7 +32,7 @@ import org.hypertrace.core.kafkastreams.framework.partitioner.GroupPartitionerBuilder; import org.hypertrace.core.kafkastreams.framework.partitioner.KeyHashPartitioner; import org.hypertrace.core.serviceframework.config.ConfigClient; -import org.hypertrace.core.spannormalizer.IpIdentity; +import org.hypertrace.core.spannormalizer.PeerIdentity; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.SpanMetadata; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -89,12 +89,13 @@ StreamsBuilder buildTopologyWithClock( Stores.persistentKeyValueStore(SPAN_STATE_STORE_NAME), keySerde, valueSerde) .withCachingEnabled(); - StoreBuilder> ipsToSpanMetadataStateStoreBuilder = - Stores.keyValueStoreBuilder( - Stores.persistentKeyValueStore(IPS_TO_SPAN_METADATA_STATE_STORE), - keySerde, - valueSerde) - .withCachingEnabled(); + StoreBuilder> + peerIdentityToSpanMetadataStateStoreBuilder = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE), + keySerde, + valueSerde) + .withCachingEnabled(); StoreBuilder>> traceEmitPunctuatorStoreBuilder = Stores.keyValueStoreBuilder( @@ -105,7 +106,7 @@ StreamsBuilder buildTopologyWithClock( streamsBuilder.addStateStore(spanStoreBuilder); streamsBuilder.addStateStore(traceStateStoreBuilder); - streamsBuilder.addStateStore(ipsToSpanMetadataStateStoreBuilder); + streamsBuilder.addStateStore(peerIdentityToSpanMetadataStateStoreBuilder); streamsBuilder.addStateStore(traceEmitPunctuatorStoreBuilder); StreamPartitioner groupPartitioner = @@ -128,7 +129,7 @@ StreamsBuilder buildTopologyWithClock( SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, TRACE_EMIT_PUNCTUATOR_STORE_NAME, - IPS_TO_SPAN_METADATA_STATE_STORE) + PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE) .to(outputTopic, outputTopicProducer); return streamsBuilder; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index a7c2fc901..c093bc8db 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -2,15 +2,14 @@ import static org.hypertrace.core.datamodel.shared.AvroBuilderCache.fastNewBuilder; import static org.hypertrace.core.kafkastreams.framework.KafkaStreamsApp.KAFKA_STREAMS_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.AGENT_TYPE_ATTRIBUTE_NAME_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DROPPED_SPANS_COUNTER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INFLIGHT_TRACE_MAX_SPAN_COUNT; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.IPS_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_CORRELATION_ENABLED_AGENTS; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SERVICE_CORRELATION_ENABLED_CUSTOMERS; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY; @@ -20,6 +19,7 @@ import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRUNCATED_TRACES_COUNTER; +import static org.hypertrace.traceenricher.enrichedspan.constants.EnrichedSpanConstants.PEER_SERVICE_NAME; import com.typesafe.config.Config; import io.micrometer.core.instrument.Counter; @@ -54,9 +54,10 @@ import org.hypertrace.core.datamodel.shared.trace.AttributeValueCreator; import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; -import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; +import org.hypertrace.core.rawspansgrouper.utils.TraceLatencyMeter; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.IpIdentity; +import org.hypertrace.core.spannormalizer.PeerIdentity; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.SpanMetadata; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -93,17 +94,17 @@ public class RawSpansProcessor private final Clock clock; private KeyValueStore spanStore; private KeyValueStore traceStateStore; - private KeyValueStore ipsToSpanMetadataStateStore; + private KeyValueStore peerIdentityToSpanMetadataStateStore; private long groupingWindowTimeoutMs; private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; private TraceEmitPunctuator traceEmitPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; - private String agentEnabledAttributeName; + private String agentTypeAttributeName; private List serviceCorrelationEnabledCustomers; - private List peerServiceNameInSpansEnabledTenants; - private RawSpansGrouperUtils rawSpansGrouperUtils; + private List peerCorrelationEnabledAgents; + private TraceLatencyMeter traceLatencyMeter; public RawSpansProcessor(Clock clock) { this.clock = clock; @@ -114,19 +115,20 @@ public void init(ProcessorContext context) { this.context = context; this.spanStore = context.getStateStore(SPAN_STATE_STORE_NAME); this.traceStateStore = context.getStateStore(TRACE_STATE_STORE); - this.ipsToSpanMetadataStateStore = context.getStateStore(IPS_TO_SPAN_METADATA_STATE_STORE); + this.peerIdentityToSpanMetadataStateStore = + context.getStateStore(PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); - this.agentEnabledAttributeName = - jobConfig.hasPath(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) - ? jobConfig.getString(AGENT_ENABLED_ATTRIBUTE_NAME_CONFIG) + this.agentTypeAttributeName = + jobConfig.hasPath(AGENT_TYPE_ATTRIBUTE_NAME_CONFIG) + ? jobConfig.getString(AGENT_TYPE_ATTRIBUTE_NAME_CONFIG) : null; this.serviceCorrelationEnabledCustomers = jobConfig.hasPath(SERVICE_CORRELATION_ENABLED_CUSTOMERS) ? jobConfig.getStringList(SERVICE_CORRELATION_ENABLED_CUSTOMERS) : Collections.emptyList(); - this.peerServiceNameInSpansEnabledTenants = - jobConfig.hasPath(PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS) - ? jobConfig.getStringList(PEER_SERVICE_NAME_IN_SPANS_ENABLED_CUSTOMERS) + this.peerCorrelationEnabledAgents = + jobConfig.hasPath(PEER_CORRELATION_ENABLED_AGENTS) + ? jobConfig.getStringList(PEER_CORRELATION_ENABLED_AGENTS) : Collections.emptyList(); this.groupingWindowTimeoutMs = jobConfig.getLong(SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY) * 1000; @@ -136,7 +138,7 @@ public void init(ProcessorContext context) { && jobConfig.getDouble(DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY) <= 100) { this.dataflowSamplingPercent = jobConfig.getDouble(DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY); } - this.rawSpansGrouperUtils = new RawSpansGrouperUtils(dataflowSamplingPercent); + this.traceLatencyMeter = new TraceLatencyMeter(dataflowSamplingPercent); if (jobConfig.hasPath(INFLIGHT_TRACE_MAX_SPAN_COUNT)) { Config subConfig = jobConfig.getConfig(INFLIGHT_TRACE_MAX_SPAN_COUNT); @@ -248,63 +250,16 @@ private void processSpanForPeerServiceNameIdentification( String tenantId = key.getTenantId(); ByteBuffer traceId = value.getTraceId(); boolean firstEntry = (traceState == null); - final Optional maybeEnvironment = - HttpSemanticConventionUtils.getEnvironmentForSpan(event); + Optional maybeEnvironment = HttpSemanticConventionUtils.getEnvironmentForSpan(event); if (SpanSemanticConventionUtils.isClientSpanForOCFormat( event.getAttributes().getAttributeMap())) { - final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); - final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); - final Optional maybePeerPort = HttpSemanticConventionUtils.getPeerPort(event); - final String serviceName = event.getServiceName(); - final IpIdentity ipIdentity = - IpIdentity.newBuilder() - .setTenantId(tenantId) - .setEnvironment(maybeEnvironment.orElse(null)) - .setHostAddr(maybeHostAddr.orElse(null)) - .setPeerAddr(maybePeerAddr.orElse(null)) - .setPeerPort(maybePeerPort.orElse(null)) - .build(); - if (IpIdentityValidator.isValid(ipIdentity)) { - ipsToSpanMetadataStateStore.put( - ipIdentity, SpanMetadata.newBuilder().setServiceName(serviceName).build()); - } + handleClientSpan(tenantId, event, maybeEnvironment.orElse(null)); } else { - final Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); - final Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); - final Optional maybeHostPort = HttpSemanticConventionUtils.getHostPort(event); - final IpIdentity ipIdentity = - IpIdentity.newBuilder() - .setTenantId(tenantId) - .setEnvironment(maybeEnvironment.orElse(null)) - .setHostAddr(maybePeerAddr.orElse(null)) - .setPeerAddr(maybeHostAddr.orElse(null)) - .setPeerPort(maybeHostPort.orElse(null)) - .build(); - if (IpIdentityValidator.isValid(ipIdentity)) { - final SpanMetadata spanMetadata = ipsToSpanMetadataStateStore.get(ipIdentity); - if (Objects.nonNull(spanMetadata)) { - // Instead of updating the span, adding a debug log to print the service correlation - logger.debug( - "Adding {} as: {} in spanId: {} with service name: {}", - PEER_SERVICE_NAME, - spanMetadata.getServiceName(), - HexUtils.getHex(event.getEventId()), - event.getServiceName()); - - if (peerServiceNameInSpansEnabledTenants.contains(tenantId) - || peerServiceNameInSpansEnabledTenants.contains(ALL)) { - event - .getAttributes() - .getAttributeMap() - .put( - PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); - } - } - } + handleServerSpan(tenantId, event, maybeEnvironment.orElse(null)); } Timestamps timestamps = - rawSpansGrouperUtils.trackEndToEndLatencyTimestamps( + traceLatencyMeter.trackEndToEndLatencyTimestamps( currentTimeMs, firstEntry ? currentTimeMs : traceState.getTraceStartTimestamp()); StructuredTrace trace = StructuredTraceBuilder.buildStructuredTraceFromRawSpans( @@ -312,23 +267,81 @@ private void processSpanForPeerServiceNameIdentification( context.forward(new Record<>(key, trace, currentTimeMs), OUTPUT_TOPIC_PRODUCER); } + // put the peer service identity and corresponding service name in state store + private void handleClientSpan(String tenantId, Event event, String environment) { + Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); + Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); + Optional maybePeerPort = HttpSemanticConventionUtils.getPeerPort(event); + String serviceName = event.getServiceName(); + PeerIdentity peerIdentity = + PeerIdentity.newBuilder() + .setIpIdentity( + IpIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(environment) + .setHostAddr(maybeHostAddr.orElse(null)) + .setPeerAddr(maybePeerAddr.orElse(null)) + .setPeerPort(maybePeerPort.orElse(null)) + .build()) + .build(); + if (PeerIdentityValidator.isValid(peerIdentity)) { + this.peerIdentityToSpanMetadataStateStore.put( + peerIdentity, SpanMetadata.newBuilder().setServiceName(serviceName).build()); + } + } + + // get the service name for that peer service identity and correlate the current span + private void handleServerSpan(String tenantId, Event event, String environment) { + Optional maybePeerAddr = HttpSemanticConventionUtils.getPeerIpAddress(event); + Optional maybeHostAddr = HttpSemanticConventionUtils.getHostIpAddress(event); + Optional maybeHostPort = HttpSemanticConventionUtils.getHostPort(event); + PeerIdentity peerIdentity = + PeerIdentity.newBuilder() + .setIpIdentity( + IpIdentity.newBuilder() + .setTenantId(tenantId) + .setEnvironment(environment) + .setHostAddr(maybePeerAddr.orElse(null)) + .setPeerAddr(maybeHostAddr.orElse(null)) + .setPeerPort(maybeHostPort.orElse(null)) + .build()) + .build(); + if (PeerIdentityValidator.isValid(peerIdentity)) { + SpanMetadata spanMetadata = this.peerIdentityToSpanMetadataStateStore.get(peerIdentity); + if (Objects.nonNull(spanMetadata)) { + // Instead of updating the span, adding a debug log to print the service correlation + logger.debug( + "Adding {} as: {} in spanId: {} with service name: {}", + PEER_SERVICE_NAME, + spanMetadata.getServiceName(), + HexUtils.getHex(event.getEventId()), + event.getServiceName()); + + event + .getAttributes() + .getAttributeMap() + .put(PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); + } + } + } + private boolean isPeerServiceNameIdentificationRequired(Event event) { - final Attributes attributes = event.getAttributes(); + Attributes attributes = event.getAttributes(); if (Objects.isNull(attributes)) { return false; } - final Map attributeMap = attributes.getAttributeMap(); + Map attributeMap = attributes.getAttributeMap(); if (Objects.isNull(attributeMap)) { return false; } - final String value = + String agentType = attributeMap - .getOrDefault(this.agentEnabledAttributeName, AttributeValue.newBuilder().build()) + .getOrDefault(this.agentTypeAttributeName, AttributeValue.newBuilder().build()) .getValue(); - return Objects.nonNull(value) - && Boolean.parseBoolean(value) + return Objects.nonNull(agentType) + && this.peerCorrelationEnabledAgents.contains(agentType) && (this.serviceCorrelationEnabledCustomers.contains(event.getCustomerId()) || this.serviceCorrelationEnabledCustomers.contains(ALL)); } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java index 62254c9e6..8a5816c5e 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java @@ -31,7 +31,7 @@ import org.hypertrace.core.kafkastreams.framework.punctuators.action.CompletedTaskResult; import org.hypertrace.core.kafkastreams.framework.punctuators.action.RescheduleTaskResult; import org.hypertrace.core.kafkastreams.framework.punctuators.action.TaskResult; -import org.hypertrace.core.rawspansgrouper.utils.RawSpansGrouperUtils; +import org.hypertrace.core.rawspansgrouper.utils.TraceLatencyMeter; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -71,7 +71,7 @@ class TraceEmitPunctuator extends AbstractThrottledPunctuator { private final KeyValueStore traceStateStore; private final String outputTopicProducer; private final long groupingWindowTimeoutMs; - private final RawSpansGrouperUtils rawSpansGrouperUtils; + private final TraceLatencyMeter traceLatencyMeter; TraceEmitPunctuator( ThrottledPunctuatorConfig throttledPunctuatorConfig, @@ -88,7 +88,7 @@ class TraceEmitPunctuator extends AbstractThrottledPunctuator { this.traceStateStore = traceStateStore; this.outputTopicProducer = outputTopicProducer; this.groupingWindowTimeoutMs = groupingWindowTimeoutMs; - this.rawSpansGrouperUtils = new RawSpansGrouperUtils(dataflowSamplingPercent); + this.traceLatencyMeter = new TraceLatencyMeter(dataflowSamplingPercent); } protected TaskResult executeTask(long punctuateTimestamp, TraceIdentity key) { @@ -151,7 +151,7 @@ private void emitTrace(TraceIdentity key, TraceState traceState) { recordSpansPerTrace(rawSpanList.size(), List.of(Tag.of("tenant_id", tenantId))); Timestamps timestamps = - this.rawSpansGrouperUtils.trackEndToEndLatencyTimestamps( + this.traceLatencyMeter.trackEndToEndLatencyTimestamps( System.currentTimeMillis(), traceState.getTraceStartTimestamp()); StructuredTrace trace = StructuredTraceBuilder.buildStructuredTraceFromRawSpans( diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeter.java similarity index 93% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeter.java index 795b39f04..2e102167e 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtils.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeter.java @@ -11,12 +11,12 @@ import org.hypertrace.core.datamodel.shared.DataflowMetricUtils; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; -public class RawSpansGrouperUtils { +public class TraceLatencyMeter { private static final Timer spansGrouperArrivalLagTimer = PlatformMetricsRegistry.registerTimer(DataflowMetricUtils.ARRIVAL_LAG, new HashMap<>()); private final double dataflowSamplingPercent; - public RawSpansGrouperUtils(double dataflowSamplingPercent) { + public TraceLatencyMeter(double dataflowSamplingPercent) { this.dataflowSamplingPercent = dataflowSamplingPercent; } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf index 893c6bcf4..77a1b0760 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf @@ -67,8 +67,5 @@ trace.emit.punctuator { frequency = ${?TRACE_EMIT_PUNCTUATOR_FREQUENCY} } -agent.enabled.attribute.name = "mirroring.enabled" - service.correlation.enabled.customers = [] - -peer.service.name.spans.enabled.customers = [] +peer.correlation.enabled.agents = [] diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index 0a2149786..5ee2d77ff 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -1,5 +1,6 @@ package org.hypertrace.core.rawspansgrouper; +import static org.hypertrace.traceenricher.enrichedspan.constants.EnrichedSpanConstants.PEER_SERVICE_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -410,7 +411,7 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { Event event = span3.getEvent(); Map attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service1)); + attributeMap.put(PEER_SERVICE_NAME, createAttribute(service1)); event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); assertEquals(event, trace.getEventList().get(0)); @@ -424,7 +425,7 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { assertEquals(1, trace.getEventList().size()); event = span2.getEvent(); attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(RawSpanGrouperConstants.PEER_SERVICE_NAME, createAttribute(service2)); + attributeMap.put(PEER_SERVICE_NAME, createAttribute(service2)); event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); assertEquals(event, trace.getEventList().get(0)); } @@ -455,8 +456,8 @@ private Event createMirroringEvent( Attributes.newBuilder() .setAttributeMap( Map.of( - "mirroring.enabled", - createAttribute("true"), + "agent.type", + createAttribute("mirror"), "deployment.environment", createAttribute("environment"), "span.kind", diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeterTest.java similarity index 70% rename from raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java rename to raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeterTest.java index ea2a26edb..0405ffad7 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/RawSpansGrouperUtilsTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/utils/TraceLatencyMeterTest.java @@ -9,16 +9,16 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -public class RawSpansGrouperUtilsTest { +public class TraceLatencyMeterTest { @Test void testTrackEndToEndLatencyTimestamps() { - RawSpansGrouperUtils rawSpansGrouperUtils = new RawSpansGrouperUtils(100); - Timestamps timestamps = rawSpansGrouperUtils.trackEndToEndLatencyTimestamps(123L, 123L); + TraceLatencyMeter traceLatencyMeter = new TraceLatencyMeter(100); + Timestamps timestamps = traceLatencyMeter.trackEndToEndLatencyTimestamps(123L, 123L); Assertions.assertNull(timestamps); - rawSpansGrouperUtils = new RawSpansGrouperUtils(0); - timestamps = rawSpansGrouperUtils.trackEndToEndLatencyTimestamps(150L, 100L); + traceLatencyMeter = new TraceLatencyMeter(0); + timestamps = traceLatencyMeter.trackEndToEndLatencyTimestamps(150L, 100L); Assertions.assertEquals( new Timestamps( Map.of( diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index ece8c22e1..16b747d0d 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -49,8 +49,6 @@ dataflow.metriccollection.sampling.percent = 10.0 trace.emit.punctuator.frequency = 15s -agent.enabled.attribute.name = "mirroring.enabled" - +agent.type.attribute.name = "agent.type" service.correlation.enabled.customers = ["*"] - -peer.service.name.spans.enabled.customers = ["*"] \ No newline at end of file +peer.correlation.enabled.agents = ["mirror"] diff --git a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl deleted file mode 100644 index f963d78ad..000000000 --- a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/IpIdentity.avdl +++ /dev/null @@ -1,10 +0,0 @@ -@namespace("org.hypertrace.core.spannormalizer") -protocol IpIdentityProtocol { - record IpIdentity { - string tenant_id; - string environment; - string host_addr; - string peer_addr; - string peer_port; - } -} diff --git a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl new file mode 100644 index 000000000..ae2527284 --- /dev/null +++ b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl @@ -0,0 +1,15 @@ +@namespace("org.hypertrace.core.spannormalizer") +protocol PeerIdentityProtocol { + record PeerIdentity { + union { null, IpIdentity } ip_identity = null; + } + + record IpIdentity { + union { null, string } tenant_id = null; + union { null, string } environment = null; + union { null, string } host_addr = null; + union { null, string } peer_addr = null; + union { null, string } peer_port = null; + } +} + From d911dd3cb38c5ecfbf4b9b52a7e70edd18675fac Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Mon, 22 Jan 2024 17:07:54 +0530 Subject: [PATCH 08/13] Addressed review comments --- .../templates/raw-spans-grouper-config.yaml | 22 ++++---- raw-spans-grouper/helm/values.yaml | 5 +- .../PeerIdentityValidator.java | 10 ---- .../RawSpanGrouperConstants.java | 7 +-- .../rawspansgrouper/RawSpansProcessor.java | 53 +++++++++---------- .../{ => validator}/IpIdentityValidator.java | 5 +- .../validator/PeerIdentityValidator.java | 16 ++++++ .../resources/configs/common/application.conf | 6 ++- .../rawspansgrouper/RawSpansGrouperTest.java | 2 + .../raw-spans-grouper/application.conf | 6 +++ .../src/main/avro/SpanMetadata.avdl | 1 + 11 files changed, 75 insertions(+), 58 deletions(-) delete mode 100644 raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{ => validator}/IpIdentityValidator.java (69%) create mode 100644 raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/PeerIdentityValidator.java diff --git a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml index e2ce9a41c..ed899891d 100644 --- a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml +++ b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml @@ -86,14 +86,16 @@ data: trace.emit.punctuator.frequency = {{ .Values.rawSpansGrouperConfig.traceEmitPunctuatorFrequency }} {{- end }} - {{- if hasKey .Values.rawSpansGrouperConfig "serviceCorrelationEnabledCustomers" }} - service.correlation.enabled.customers = {{ .Values.rawSpansGrouperConfig.serviceCorrelationEnabledCustomers | toJson }} - {{- end }} - - {{- if hasKey .Values.rawSpansGrouperConfig "peerCorrelationEnabledAgents" }} - peer.correlation.enabled.agents = {{ .Values.rawSpansGrouperConfig.peerCorrelationEnabledAgents | toJson }} - {{- end }} - - {{- if hasKey .Values.rawSpansGrouperConfig "agentTypeAttributeName" }} - agent.type.attribute.name = {{ .Values.rawSpansGrouperConfig.agentTypeAttributeName }} + {{- if hasKey .Values.rawSpansGrouperConfig "peerCorrelation" }} + peer.correlation = { + {{- if hasKey .Values.rawSpansGrouperConfig.peerCorrelation "enabledCustomers" }} + enabled.customers = {{ .Values.rawSpansGrouperConfig.peerCorrelation.enabledCustomers | toJson }} + {{- end }} + {{- if hasKey .Values.rawSpansGrouperConfig.peerCorrelation "enabledAgents" }} + enabled.agents = {{ .Values.rawSpansGrouperConfig.peerCorrelation.enabledAgents | toJson }} + {{- end }} + {{- if hasKey .Values.rawSpansGrouperConfig.peerCorrelation "agentTypeAttribute" }} + agent.type.attribute = {{ .Values.rawSpansGrouperConfig.peerCorrelation.agentTypeAttribute }} + {{- end }} + } {{- end }} diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index ed0bea666..05ec7d95e 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -133,8 +133,9 @@ rawSpansGrouperConfig: groupPartitionerConfigServiceHost: config-service groupPartitionerConfigServicePort: 50104 traceEmitPunctuatorFrequency: 5s - serviceCorrelationEnabledCustomers: [] - peerCorrelationEnabledAgents: [] + peerCorrelation: + enabledCustomers: [] + enabledAgents: [] span: groupby: diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java deleted file mode 100644 index 3941967ae..000000000 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/PeerIdentityValidator.java +++ /dev/null @@ -1,10 +0,0 @@ -package org.hypertrace.core.rawspansgrouper; - -import org.hypertrace.core.spannormalizer.PeerIdentity; - -public class PeerIdentityValidator { - - public static boolean isValid(PeerIdentity peerIdentity) { - return IpIdentityValidator.isValid(peerIdentity.getIpIdentity()); - } -} diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java index 780992543..0ff299331 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpanGrouperConstants.java @@ -15,9 +15,10 @@ public class RawSpanGrouperConstants { public static final String TRACE_CREATION_TIME = "trace.creation.time"; public static final String DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY = "dataflow.metriccollection.sampling.percent"; - public static final String AGENT_TYPE_ATTRIBUTE_NAME_CONFIG = "agent.type.attribute.name"; - public static final String SERVICE_CORRELATION_ENABLED_CUSTOMERS = - "service.correlation.enabled.customers"; + public static final String PEER_CORRELATION_AGENT_TYPE_ATTRIBUTE_CONFIG = + "peer.correlation.agent.type.attribute"; + public static final String PEER_CORRELATION_ENABLED_CUSTOMERS = + "peer.correlation.enabled.customers"; public static final String PEER_CORRELATION_ENABLED_AGENTS = "peer.correlation.enabled.agents"; public static final String INFLIGHT_TRACE_MAX_SPAN_COUNT = "max.span.count"; public static final String DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT = "default.max.span.count"; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index c093bc8db..72df5ce69 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -2,16 +2,16 @@ import static org.hypertrace.core.datamodel.shared.AvroBuilderCache.fastNewBuilder; import static org.hypertrace.core.kafkastreams.framework.KafkaStreamsApp.KAFKA_STREAMS_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.AGENT_TYPE_ATTRIBUTE_NAME_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DATAFLOW_SAMPLING_PERCENT_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.DROPPED_SPANS_COUNTER; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.INFLIGHT_TRACE_MAX_SPAN_COUNT; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_CORRELATION_AGENT_TYPE_ATTRIBUTE_CONFIG; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_CORRELATION_ENABLED_AGENTS; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_CORRELATION_ENABLED_CUSTOMERS; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SERVICE_CORRELATION_ENABLED_CUSTOMERS; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.SPAN_STATE_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR; @@ -45,7 +45,6 @@ import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.AttributeValue; -import org.hypertrace.core.datamodel.Attributes; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -55,6 +54,7 @@ import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; import org.hypertrace.core.rawspansgrouper.utils.TraceLatencyMeter; +import org.hypertrace.core.rawspansgrouper.validator.PeerIdentityValidator; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.IpIdentity; import org.hypertrace.core.spannormalizer.PeerIdentity; @@ -101,8 +101,8 @@ public class RawSpansProcessor private long defaultMaxSpanCountLimit = Long.MAX_VALUE; private TraceEmitPunctuator traceEmitPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; - private String agentTypeAttributeName; - private List serviceCorrelationEnabledCustomers; + private String peerCorrelationAgentTypeAttribute; + private List peerCorrelationEnabledCustomers; private List peerCorrelationEnabledAgents; private TraceLatencyMeter traceLatencyMeter; @@ -118,13 +118,13 @@ public void init(ProcessorContext context) { this.peerIdentityToSpanMetadataStateStore = context.getStateStore(PEER_IDENTITY_TO_SPAN_METADATA_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); - this.agentTypeAttributeName = - jobConfig.hasPath(AGENT_TYPE_ATTRIBUTE_NAME_CONFIG) - ? jobConfig.getString(AGENT_TYPE_ATTRIBUTE_NAME_CONFIG) + this.peerCorrelationAgentTypeAttribute = + jobConfig.hasPath(PEER_CORRELATION_AGENT_TYPE_ATTRIBUTE_CONFIG) + ? jobConfig.getString(PEER_CORRELATION_AGENT_TYPE_ATTRIBUTE_CONFIG) : null; - this.serviceCorrelationEnabledCustomers = - jobConfig.hasPath(SERVICE_CORRELATION_ENABLED_CUSTOMERS) - ? jobConfig.getStringList(SERVICE_CORRELATION_ENABLED_CUSTOMERS) + this.peerCorrelationEnabledCustomers = + jobConfig.hasPath(PEER_CORRELATION_ENABLED_CUSTOMERS) + ? jobConfig.getStringList(PEER_CORRELATION_ENABLED_CUSTOMERS) : Collections.emptyList(); this.peerCorrelationEnabledAgents = jobConfig.hasPath(PEER_CORRELATION_ENABLED_AGENTS) @@ -286,7 +286,11 @@ private void handleClientSpan(String tenantId, Event event, String environment) .build(); if (PeerIdentityValidator.isValid(peerIdentity)) { this.peerIdentityToSpanMetadataStateStore.put( - peerIdentity, SpanMetadata.newBuilder().setServiceName(serviceName).build()); + peerIdentity, + SpanMetadata.newBuilder() + .setServiceName(serviceName) + .setEventId(HexUtils.getHex(event.getEventId())) + .build()); } } @@ -309,11 +313,11 @@ private void handleServerSpan(String tenantId, Event event, String environment) if (PeerIdentityValidator.isValid(peerIdentity)) { SpanMetadata spanMetadata = this.peerIdentityToSpanMetadataStateStore.get(peerIdentity); if (Objects.nonNull(spanMetadata)) { - // Instead of updating the span, adding a debug log to print the service correlation logger.debug( - "Adding {} as: {} in spanId: {} with service name: {}", + "Adding {} as: {} from spanId: {} in spanId: {} with service name: {}", PEER_SERVICE_NAME, spanMetadata.getServiceName(), + spanMetadata.getEventId(), HexUtils.getHex(event.getEventId()), event.getServiceName()); @@ -326,24 +330,17 @@ private void handleServerSpan(String tenantId, Event event, String environment) } private boolean isPeerServiceNameIdentificationRequired(Event event) { - Attributes attributes = event.getAttributes(); - if (Objects.isNull(attributes)) { - return false; - } - - Map attributeMap = attributes.getAttributeMap(); - if (Objects.isNull(attributeMap)) { - return false; - } - String agentType = - attributeMap - .getOrDefault(this.agentTypeAttributeName, AttributeValue.newBuilder().build()) + event + .getAttributes() + .getAttributeMap() + .getOrDefault( + this.peerCorrelationAgentTypeAttribute, AttributeValue.newBuilder().build()) .getValue(); return Objects.nonNull(agentType) && this.peerCorrelationEnabledAgents.contains(agentType) - && (this.serviceCorrelationEnabledCustomers.contains(event.getCustomerId()) - || this.serviceCorrelationEnabledCustomers.contains(ALL)); + && (this.peerCorrelationEnabledCustomers.contains(event.getCustomerId()) + || this.peerCorrelationEnabledCustomers.contains(ALL)); } private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/IpIdentityValidator.java similarity index 69% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/IpIdentityValidator.java index c7059118d..09a210979 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/IpIdentityValidator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/IpIdentityValidator.java @@ -1,4 +1,4 @@ -package org.hypertrace.core.rawspansgrouper; +package org.hypertrace.core.rawspansgrouper.validator; import java.util.Objects; import org.hypertrace.core.spannormalizer.IpIdentity; @@ -10,7 +10,6 @@ public static boolean isValid(IpIdentity ipIdentity) { } return Objects.nonNull(ipIdentity.getHostAddr()) - || Objects.nonNull(ipIdentity.getPeerAddr()) - || Objects.nonNull(ipIdentity.getPeerPort()); + || (Objects.nonNull(ipIdentity.getPeerAddr()) && Objects.nonNull(ipIdentity.getPeerPort())); } } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/PeerIdentityValidator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/PeerIdentityValidator.java new file mode 100644 index 000000000..ce64f1aff --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/validator/PeerIdentityValidator.java @@ -0,0 +1,16 @@ +package org.hypertrace.core.rawspansgrouper.validator; + +import java.util.Objects; +import org.hypertrace.core.spannormalizer.IpIdentity; +import org.hypertrace.core.spannormalizer.PeerIdentity; + +public class PeerIdentityValidator { + + public static boolean isValid(PeerIdentity peerIdentity) { + final IpIdentity ipIdentity = peerIdentity.getIpIdentity(); + if (Objects.nonNull(ipIdentity)) { + return IpIdentityValidator.isValid(ipIdentity); + } + return true; + } +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf index 77a1b0760..632690f41 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/main/resources/configs/common/application.conf @@ -67,5 +67,7 @@ trace.emit.punctuator { frequency = ${?TRACE_EMIT_PUNCTUATOR_FREQUENCY} } -service.correlation.enabled.customers = [] -peer.correlation.enabled.agents = [] +peer.correlation = { + enabled.customers = [] + enabled.agents = [] +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index 5ee2d77ff..88826ab40 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -13,6 +13,7 @@ import java.nio.ByteBuffer; import java.nio.file.Path; import java.time.Clock; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -434,6 +435,7 @@ private Event createEvent(String eventId, String tenantId) { return Event.newBuilder() .setCustomerId(tenantId) .setEventId(ByteBuffer.wrap(eventId.getBytes())) + .setAttributes(Attributes.newBuilder().setAttributeMap(Collections.emptyMap()).build()) .setStartTimeMillis(System.currentTimeMillis()) .build(); } diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index 16b747d0d..490292279 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -49,6 +49,12 @@ dataflow.metriccollection.sampling.percent = 10.0 trace.emit.punctuator.frequency = 15s +peer.correlation = { + enabled.customers = ["*"] + enabled.agents = ["mirror"] + agent.type.attribute = "agent.type" +} + agent.type.attribute.name = "agent.type" service.correlation.enabled.customers = ["*"] peer.correlation.enabled.agents = ["mirror"] diff --git a/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl index 0961391db..69fac505d 100644 --- a/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl +++ b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl @@ -2,5 +2,6 @@ protocol SpanMetadataProtocol { record SpanMetadata { string service_name; + string event_id; } } From 4a7b6d8609943b6343eed75300f9e97e2e87aa69 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Tue, 23 Jan 2024 14:22:52 +0530 Subject: [PATCH 09/13] updates --- .../rawspansgrouper/RawSpansProcessor.java | 11 ++++------- .../rawspansgrouper/RawSpansGrouperTest.java | 19 ++++++++++++------- 2 files changed, 16 insertions(+), 14 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 72df5ce69..22d058b0f 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -44,12 +44,12 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.state.KeyValueStore; -import org.hypertrace.core.datamodel.AttributeValue; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.Timestamps; import org.hypertrace.core.datamodel.shared.HexUtils; +import org.hypertrace.core.datamodel.shared.SpanAttributeUtils; import org.hypertrace.core.datamodel.shared.trace.AttributeValueCreator; import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; @@ -322,7 +322,7 @@ private void handleServerSpan(String tenantId, Event event, String environment) event.getServiceName()); event - .getAttributes() + .getEnrichedAttributes() .getAttributeMap() .put(PEER_SERVICE_NAME, AttributeValueCreator.create(spanMetadata.getServiceName())); } @@ -331,11 +331,8 @@ private void handleServerSpan(String tenantId, Event event, String environment) private boolean isPeerServiceNameIdentificationRequired(Event event) { String agentType = - event - .getAttributes() - .getAttributeMap() - .getOrDefault( - this.peerCorrelationAgentTypeAttribute, AttributeValue.newBuilder().build()) + SpanAttributeUtils.getAttributeValueWithDefault( + event, this.peerCorrelationAgentTypeAttribute, null) .getValue(); return Objects.nonNull(agentType) && this.peerCorrelationEnabledAgents.contains(agentType) diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java index 88826ab40..53ec27027 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java @@ -410,10 +410,10 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { trace = outputTopic.readValue(); assertEquals(1, trace.getEventList().size()); Event event = span3.getEvent(); - Map attributeMap = - new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(PEER_SERVICE_NAME, createAttribute(service1)); - event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + event.setEnrichedAttributes( + Attributes.newBuilder() + .setAttributeMap(Map.of(PEER_SERVICE_NAME, createAttribute(service1))) + .build()); assertEquals(event, trace.getEventList().get(0)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-4"), span4); @@ -425,9 +425,10 @@ void testMirroringSpansGrouping(@TempDir Path tempDir) { trace = outputTopic.readValue(); assertEquals(1, trace.getEventList().size()); event = span2.getEvent(); - attributeMap = new HashMap<>(event.getAttributes().getAttributeMap()); - attributeMap.put(PEER_SERVICE_NAME, createAttribute(service2)); - event.setAttributes(Attributes.newBuilder().setAttributeMap(attributeMap).build()); + event.setEnrichedAttributes( + Attributes.newBuilder() + .setAttributeMap(Map.of(PEER_SERVICE_NAME, createAttribute(service2))) + .build()); assertEquals(event, trace.getEventList().get(0)); } @@ -436,6 +437,8 @@ private Event createEvent(String eventId, String tenantId) { .setCustomerId(tenantId) .setEventId(ByteBuffer.wrap(eventId.getBytes())) .setAttributes(Attributes.newBuilder().setAttributeMap(Collections.emptyMap()).build()) + .setEnrichedAttributes( + Attributes.newBuilder().setAttributeMap(Collections.emptyMap()).build()) .setStartTimeMillis(System.currentTimeMillis()) .build(); } @@ -454,6 +457,8 @@ private Event createMirroringEvent( .setServiceName(service) .setEventId(ByteBuffer.wrap(eventId.getBytes())) .setStartTimeMillis(System.currentTimeMillis()) + .setEnrichedAttributes( + Attributes.newBuilder().setAttributeMap(Collections.emptyMap()).build()) .setAttributes( Attributes.newBuilder() .setAttributeMap( From aed68f227e13e6914c295a2f789818410e454f08 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Tue, 23 Jan 2024 14:26:43 +0530 Subject: [PATCH 10/13] updates --- .../hypertrace/core/rawspansgrouper/RawSpansProcessor.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 22d058b0f..61cb9fd69 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -331,9 +331,8 @@ private void handleServerSpan(String tenantId, Event event, String environment) private boolean isPeerServiceNameIdentificationRequired(Event event) { String agentType = - SpanAttributeUtils.getAttributeValueWithDefault( - event, this.peerCorrelationAgentTypeAttribute, null) - .getValue(); + SpanAttributeUtils.getStringAttributeWithDefault( + event, this.peerCorrelationAgentTypeAttribute, null); return Objects.nonNull(agentType) && this.peerCorrelationEnabledAgents.contains(agentType) && (this.peerCorrelationEnabledCustomers.contains(event.getCustomerId()) From 131029f3c300dd55035f932300236f0f65b2298b Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Wed, 24 Jan 2024 12:47:06 +0530 Subject: [PATCH 11/13] updates --- .../test/resources/configs/raw-spans-grouper/application.conf | 4 ---- .../src/main/avro/trace-identity/PeerIdentity.avdl | 1 - 2 files changed, 5 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf index 490292279..d7dc1d302 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf +++ b/raw-spans-grouper/raw-spans-grouper/src/test/resources/configs/raw-spans-grouper/application.conf @@ -54,7 +54,3 @@ peer.correlation = { enabled.agents = ["mirror"] agent.type.attribute = "agent.type" } - -agent.type.attribute.name = "agent.type" -service.correlation.enabled.customers = ["*"] -peer.correlation.enabled.agents = ["mirror"] diff --git a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl index ae2527284..e33ca0f02 100644 --- a/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl +++ b/span-normalizer/span-normalizer-api/src/main/avro/trace-identity/PeerIdentity.avdl @@ -12,4 +12,3 @@ protocol PeerIdentityProtocol { union { null, string } peer_port = null; } } - From 02cbae2eca7ef9c826aa14f1fc2ac35b319e4317 Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Mon, 29 Jan 2024 15:11:21 +0530 Subject: [PATCH 12/13] Addressed review comments --- .../constants/utils/EnrichedSpanUtils.java | 2 +- .../core/rawspansgrouper/RawSpansProcessor.java | 7 +++++-- .../utils/http/HttpSemanticConventionUtils.java | 11 +++++++++++ .../src/main/avro/SpanMetadata.avdl | 2 +- 4 files changed, 18 insertions(+), 4 deletions(-) diff --git a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java index 38f0cffd8..a55f63ee0 100644 --- a/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java +++ b/hypertrace-trace-enricher/enriched-span-constants/src/main/java/org/hypertrace/traceenricher/enrichedspan/constants/utils/EnrichedSpanUtils.java @@ -350,7 +350,7 @@ public static Optional getFullHttpUrl(Event event) { } public static Optional getDestinationIpAddress(Event event) { - return HttpSemanticConventionUtils.getPeerIpAddress(event); + return HttpSemanticConventionUtils.getDestinationIpAddress(event); } public static Optional getPath(Event event) { diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 61cb9fd69..15b321ac5 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -197,6 +197,8 @@ public void process(Record record) { } Event event = value.getEvent(); + // spans whose trace is not created by ByPass flow, their trace will be created in below + // function call. Those spans' trace will not be created by TraceEmitPunctuator if (isPeerServiceNameIdentificationRequired(event)) { processSpanForPeerServiceNameIdentification(key, value, traceState, currentTimeMs); return; @@ -258,6 +260,7 @@ private void processSpanForPeerServiceNameIdentification( handleServerSpan(tenantId, event, maybeEnvironment.orElse(null)); } + // create structured trace and forward Timestamps timestamps = traceLatencyMeter.trackEndToEndLatencyTimestamps( currentTimeMs, firstEntry ? currentTimeMs : traceState.getTraceStartTimestamp()); @@ -289,7 +292,7 @@ private void handleClientSpan(String tenantId, Event event, String environment) peerIdentity, SpanMetadata.newBuilder() .setServiceName(serviceName) - .setEventId(HexUtils.getHex(event.getEventId())) + .setEventId(event.getEventId()) .build()); } } @@ -317,7 +320,7 @@ private void handleServerSpan(String tenantId, Event event, String environment) "Adding {} as: {} from spanId: {} in spanId: {} with service name: {}", PEER_SERVICE_NAME, spanMetadata.getServiceName(), - spanMetadata.getEventId(), + HexUtils.getHex(spanMetadata.getEventId()), HexUtils.getHex(event.getEventId()), event.getServiceName()); diff --git a/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java b/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java index 5021273f3..3a25a1386 100644 --- a/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java +++ b/semantic-convention-utils/src/main/java/org/hypertrace/semantic/convention/utils/http/HttpSemanticConventionUtils.java @@ -212,6 +212,17 @@ public static Optional getFullHttpUrl(Event event) { : getValidHttpUrl(event).map(AttributeValue::getValue); } + public static Optional getDestinationIpAddress(Event event) { + return Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_SOCK_PEER_ADDR.getValue())) + .or( + () -> + Optional.ofNullable( + SpanAttributeUtils.getStringAttribute( + event, OTelSpanSemanticConventions.NET_PEER_IP.getValue()))); + } + public static Optional getPeerIpAddress(Event event) { return Optional.ofNullable( SpanAttributeUtils.getStringAttribute( diff --git a/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl index 69fac505d..2eee5da02 100644 --- a/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl +++ b/span-normalizer/span-normalizer-api/src/main/avro/SpanMetadata.avdl @@ -2,6 +2,6 @@ protocol SpanMetadataProtocol { record SpanMetadata { string service_name; - string event_id; + bytes event_id; } } From 0912315b52c370633d91920518d8515025bedb5f Mon Sep 17 00:00:00 2001 From: Sanket Mundra Date: Mon, 29 Jan 2024 15:20:18 +0530 Subject: [PATCH 13/13] update --- .../core/rawspansgrouper/RawSpansProcessor.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 15b321ac5..ab561bc8a 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -333,13 +333,15 @@ private void handleServerSpan(String tenantId, Event event, String environment) } private boolean isPeerServiceNameIdentificationRequired(Event event) { + if (!this.peerCorrelationEnabledCustomers.contains(event.getCustomerId()) + && !this.peerCorrelationEnabledCustomers.contains(ALL)) { + return false; + } + String agentType = SpanAttributeUtils.getStringAttributeWithDefault( event, this.peerCorrelationAgentTypeAttribute, null); - return Objects.nonNull(agentType) - && this.peerCorrelationEnabledAgents.contains(agentType) - && (this.peerCorrelationEnabledCustomers.contains(event.getCustomerId()) - || this.peerCorrelationEnabledCustomers.contains(ALL)); + return Objects.nonNull(agentType) && this.peerCorrelationEnabledAgents.contains(agentType); } private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) {