From f3e44375020a0eccec983223875cd0eca48b2f50 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Wed, 4 Oct 2023 16:38:00 +0530 Subject: [PATCH 01/31] wrap up --- gradle/libs.versions.toml | 2 +- .../RawSpanGrouperConstants.java | 8 + .../core/rawspansgrouper/RawSpansGrouper.java | 14 +- .../rawspansgrouper/RawSpansProcessor.java | 120 ++++------ ...Punctuator.java => TraceEmitCallback.java} | 209 +++++++----------- .../resources/configs/common/application.conf | 9 + .../rawspansgrouper/RawSpansGrouperTest.java | 8 +- .../TraceEmitCallbackTest.java | 103 +++++++++ .../TraceEmitPunctuatorTest.java | 117 ---------- .../raw-spans-grouper/application.conf | 6 + .../src/main/avro/TraceState.avdl | 2 +- 11 files changed, 276 insertions(+), 322 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{TraceEmitPunctuator.java => TraceEmitCallback.java} (54%) create mode 100644 raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java delete mode 100644 raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f5ee293c5..61f54f91a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -3,7 +3,7 @@ hypertrace-entity-service = "0.8.78" hypertrace-config-service = "0.1.54" hypertrace-grpc-utils = "0.12.4" hypertrace-serviceFramework = "0.1.60" -hypertrace-kafkaStreams = "0.3.8" +hypertrace-kafkaStreams = "0.4.0-SNAPSHOT" hypertrace-view-generator = "0.4.19" grpc = "1.57.2" 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..f89672d21 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 @@ -17,4 +17,12 @@ 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"; + static final String TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME = + "trace-emit-callback-registry-store"; + static final String TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY = + "trace.emit.callback.registry.frequency"; + static final String TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY = + "trace.emit.callback.registry.yield"; + static final String TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY = + "trace.emit.callback.registry.window"; } 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..85116b14e 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 @@ -5,12 +5,15 @@ 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_STATE_STORE_NAME; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_STATE_STORE; import com.typesafe.config.Config; +import java.util.ArrayList; import java.util.List; import java.util.Map; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; @@ -72,8 +75,16 @@ public StreamsBuilder buildTopology( Stores.persistentKeyValueStore(SPAN_STATE_STORE_NAME), keySerde, valueSerde) .withCachingEnabled(); + StoreBuilder> traceEmitCallbackRegistryStoreBuilder = + Stores.keyValueStoreBuilder( + Stores.persistentKeyValueStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME), + Serdes.Long(), + Serdes.ListSerde(ArrayList.class, valueSerde)) + .withCachingEnabled(); + streamsBuilder.addStateStore(spanStoreBuilder); streamsBuilder.addStateStore(traceStateStoreBuilder); + streamsBuilder.addStateStore(traceEmitCallbackRegistryStoreBuilder); StreamPartitioner groupPartitioner = new GroupPartitionerBuilder() @@ -93,7 +104,8 @@ public StreamsBuilder buildTopology( RawSpansProcessor::new, Named.as(RawSpansProcessor.class.getSimpleName()), SPAN_STATE_STORE_NAME, - TRACE_STATE_STORE) + TRACE_STATE_STORE, + TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME) .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 388765fcf..f1dd85fd7 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 @@ -9,6 +9,10 @@ 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; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_STATE_STORE; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRUNCATED_TRACES_COUNTER; @@ -31,11 +35,12 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; 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.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.shared.HexUtils; +import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuator; +import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -44,11 +49,11 @@ import org.slf4j.LoggerFactory; /** - * Receives spans keyed by trace_id and stores them. A {@link TraceEmitPunctuator} is scheduled to - * run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If - * any spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} - * interval then the {@link RawSpansProcessor#groupingWindowTimeoutMs} will get reset and the trace - * will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} time to accept spans. + * Receives spans keyed by trace_id and stores them. A {@link TraceEmitCallback} is scheduled to run + * after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If any + * spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval + * then the trace will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} time to + * accept spans. */ public class RawSpansProcessor implements Transformer> { @@ -58,6 +63,13 @@ public class RawSpansProcessor "hypertrace.rawspansgrouper.processing.latency"; private static final ConcurrentMap tenantToSpansGroupingTimer = new ConcurrentHashMap<>(); + // counter for number of spans dropped per tenant + private static final ConcurrentMap droppedSpansCounter = + new ConcurrentHashMap<>(); + + // counter for number of truncated traces per tenant + private static final ConcurrentMap truncatedTracesCounter = + new ConcurrentHashMap<>(); private ProcessorContext context; private KeyValueStore spanStore; private KeyValueStore traceStateStore; @@ -66,14 +78,8 @@ public class RawSpansProcessor private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; - - // counter for number of spans dropped per tenant - private static final ConcurrentMap droppedSpansCounter = - new ConcurrentHashMap<>(); - - // counter for number of truncated traces per tenant - private static final ConcurrentMap truncatedTracesCounter = - new ConcurrentHashMap<>(); + private CallbackRegistryPunctuator traceEmitCallbackRegistry; + private Cancellable traceEmitCallbackRegistryCancellable; @Override public void init(ProcessorContext context) { @@ -106,7 +112,29 @@ public void init(ProcessorContext context) { } this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); - restorePunctuators(); + + TraceEmitCallback traceEmitCallback = + new TraceEmitCallback( + context, + spanStore, + traceStateStore, + outputTopic, + groupingWindowTimeoutMs, + dataflowSamplingPercent); + KeyValueStore> traceEmitCallbackRegistryStore = + context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); + traceEmitCallbackRegistry = + new CallbackRegistryPunctuator<>( + new CallbackRegistryPunctuatorConfig( + jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY).toMillis(), + jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY).toMillis()), + traceEmitCallbackRegistryStore, + traceEmitCallback::callback); + traceEmitCallbackRegistryCancellable = + context.schedule( + jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), + PunctuationType.WALL_CLOCK_TIME, + traceEmitCallbackRegistry); } public KeyValue transform(TraceIdentity key, RawSpan value) { @@ -125,35 +153,20 @@ public KeyValue transform(TraceIdentity key, Raw ByteBuffer spanId = value.getEvent().getEventId(); spanStore.put(new SpanIdentity(tenantId, traceId, spanId), value); - /* - the trace emit ts is essentially currentTs + groupingWindowTimeoutMs - i.e. if there is no span added in the next 'groupingWindowTimeoutMs' interval - then the trace can be finalized and emitted - */ - long traceEmitTs = currentTimeMs + groupingWindowTimeoutMs; - if (logger.isDebugEnabled()) { - logger.debug( - "Updating trigger_ts=[{}] for for tenant_id=[{}], trace_id=[{}]", - Instant.ofEpochMilli(traceEmitTs), - key.getTenantId(), - HexUtils.getHex(traceId)); - } - if (firstEntry) { traceState = fastNewBuilder(TraceState.Builder.class) .setTraceStartTimestamp(currentTimeMs) .setTraceEndTimestamp(currentTimeMs) - .setEmitTs(traceEmitTs) + .setEmitTs(-1) // deprecated, not used anymore .setTenantId(tenantId) .setTraceId(traceId) .setSpanIds(List.of(spanId)) .build(); - schedulePunctuator(key); + traceEmitCallbackRegistry.add(currentTimeMs, key); } else { traceState.getSpanIds().add(spanId); traceState.setTraceEndTimestamp(currentTimeMs); - traceState.setEmitTs(traceEmitTs); } traceStateStore.put(key, traceState); @@ -213,47 +226,8 @@ private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { return false; } - private void schedulePunctuator(TraceIdentity key) { - TraceEmitPunctuator punctuator = - new TraceEmitPunctuator( - key, - context, - spanStore, - traceStateStore, - outputTopic, - groupingWindowTimeoutMs, - dataflowSamplingPercent); - Cancellable cancellable = - context.schedule( - Duration.ofMillis(groupingWindowTimeoutMs), - PunctuationType.WALL_CLOCK_TIME, - punctuator); - punctuator.setCancellable(cancellable); - logger.debug( - "Scheduled a punctuator to emit trace for key=[{}] to run after [{}] ms", - key, - groupingWindowTimeoutMs); - } - @Override - public void close() {} - - /** - * Punctuators are not persisted - so on restart we recover punctuators and schedule them to run - * after {@link RawSpansProcessor#groupingWindowTimeoutMs} - */ - void restorePunctuators() { - long count = 0; - Instant start = Instant.now(); - try (KeyValueIterator it = traceStateStore.all()) { - while (it.hasNext()) { - schedulePunctuator(it.next().key); - count++; - } - logger.info( - "Restored=[{}] punctuators, Duration=[{}]", - count, - Duration.between(start, Instant.now())); - } + public void close() { + traceEmitCallbackRegistryCancellable.cancel(); } } 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/TraceEmitCallback.java similarity index 54% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallback.java index 0c0cae52a..959f47ed9 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/TraceEmitCallback.java @@ -9,8 +9,6 @@ import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.Timer; import java.nio.ByteBuffer; -import java.time.Duration; -import java.time.Instant; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -20,10 +18,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import org.apache.kafka.streams.processor.Cancellable; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.PunctuationType; -import org.apache.kafka.streams.processor.Punctuator; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.RawSpan; @@ -33,6 +28,9 @@ 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.kafkastreams.framework.callbacks.action.CallbackAction; +import org.hypertrace.core.kafkastreams.framework.callbacks.action.DropCallbackAction; +import org.hypertrace.core.kafkastreams.framework.callbacks.action.RescheduleCallbackAction; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -41,12 +39,12 @@ import org.slf4j.LoggerFactory; /** - * Checks if a trace can be finalized and emitted based on inactivity period of {@link + * Callback to check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansProcessor#groupingWindowTimeoutMs} */ -class TraceEmitPunctuator implements Punctuator { +class TraceEmitCallback { - private static final Logger logger = LoggerFactory.getLogger(TraceEmitPunctuator.class); + private static final Logger logger = LoggerFactory.getLogger(TraceEmitCallback.class); private static final Object mutex = new Object(); private static final Timer spansGrouperArrivalLagTimer = @@ -70,24 +68,21 @@ class TraceEmitPunctuator implements Punctuator { private static final ConcurrentMap tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); + private static final DropCallbackAction DROP_CALLBACK_ACTION = new DropCallbackAction(); 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 Cancellable cancellable; - TraceEmitPunctuator( - TraceIdentity key, + TraceEmitCallback( ProcessorContext context, KeyValueStore spanStore, KeyValueStore traceStateStore, To outputTopicProducer, long groupingWindowTimeoutMs, double dataflowSamplingPercent) { - this.key = key; this.context = context; this.spanStore = spanStore; this.traceStateStore = traceStateStore; @@ -96,143 +91,107 @@ class TraceEmitPunctuator implements Punctuator { this.dataflowSamplingPercent = dataflowSamplingPercent; } - public void setCancellable(Cancellable cancellable) { - this.cancellable = cancellable; - } - - /** - * @param timestamp correspond to current system time - */ - @Override - public void punctuate(long timestamp) { - Instant startTime = Instant.now(); - // always cancel the punctuator else it will get re-scheduled automatically - cancellable.cancel(); - + CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { TraceState traceState = traceStateStore.get(key); if (null == traceState || null == traceState.getSpanIds() || traceState.getSpanIds().isEmpty()) { - /* - todo - debug why this happens . - Typically seen when punctuators are created via {@link RawSpansGroupingTransformer.restorePunctuators} - */ logger.warn( "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", key.getTenantId(), HexUtils.getHex(key.getTraceId())); - return; + return DROP_CALLBACK_ACTION; } - - long emitTs = traceState.getEmitTs(); - if (emitTs <= timestamp) { - // we can emit this trace so just delete the entry for this 'key' + if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { // Implies that no new spans for the trace have arrived within the last // 'groupingWindowTimeoutMs' interval // so the trace can be finalized and emitted - traceStateStore.delete(key); - - ByteBuffer traceId = traceState.getTraceId(); - String tenantId = traceState.getTenantId(); - List rawSpanList = new ArrayList<>(); - - Set spanIds = new HashSet<>(traceState.getSpanIds()); - spanIds.forEach( - v -> { - SpanIdentity spanIdentity = new SpanIdentity(tenantId, traceId, v); - RawSpan rawSpan = spanStore.delete(spanIdentity); - // ideally this shouldn't happen - if (rawSpan != null) { - rawSpanList.add(rawSpan); - } - }); + emitTrace(key, traceState); + // no need of running again for this + return DROP_CALLBACK_ACTION; + } + return new RescheduleCallbackAction( + traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); + } - if (traceState.getSpanIds().size() != spanIds.size()) { - tenantToTraceWithDuplicateSpansCounter - .computeIfAbsent( - tenantId, - k -> - PlatformMetricsRegistry.registerCounter( - TRACE_WITH_DUPLICATE_SPANS, Map.of("tenantId", k))) - .increment(); - if (logger.isDebugEnabled()) { - logger.debug( - "Duplicate spanIds: [{}], unique spanIds count: [{}] for tenant: [{}] trace: [{}]", - traceState.getSpanIds().size(), - spanIds.size(), + private void emitTrace(TraceIdentity key, TraceState traceState) { + traceStateStore.delete(key); + + ByteBuffer traceId = traceState.getTraceId(); + String tenantId = traceState.getTenantId(); + List rawSpanList = new ArrayList<>(); + + Set spanIds = new HashSet<>(traceState.getSpanIds()); + spanIds.forEach( + v -> { + SpanIdentity spanIdentity = new SpanIdentity(tenantId, traceId, v); + RawSpan rawSpan = spanStore.delete(spanIdentity); + // ideally this shouldn't happen + if (rawSpan != null) { + rawSpanList.add(rawSpan); + } + }); + + if (traceState.getSpanIds().size() != spanIds.size()) { + tenantToTraceWithDuplicateSpansCounter + .computeIfAbsent( tenantId, - HexUtils.getHex(traceId)); - } - } - - recordSpansPerTrace(rawSpanList.size(), List.of(Tag.of("tenant_id", tenantId))); - Timestamps timestamps = - trackEndToEndLatencyTimestamps(timestamp, traceState.getTraceStartTimestamp()); - StructuredTrace trace = - StructuredTraceBuilder.buildStructuredTraceFromRawSpans( - rawSpanList, traceId, tenantId, timestamps); - + k -> + PlatformMetricsRegistry.registerCounter( + TRACE_WITH_DUPLICATE_SPANS, Map.of("tenantId", k))) + .increment(); if (logger.isDebugEnabled()) { logger.debug( - "Emit tenant_id=[{}], trace_id=[{}], spans_count=[{}]", + "Duplicate spanIds: [{}], unique spanIds count: [{}] for tenant: [{}] trace: [{}]", + traceState.getSpanIds().size(), + spanIds.size(), tenantId, - HexUtils.getHex(traceId), - rawSpanList.size()); + HexUtils.getHex(traceId)); } + } - // report entries in spanStore - if (spanStoreCountRateLimiter.tryAcquire()) { - tenantToSpanStoreCountCounter - .computeIfAbsent( - tenantId, - k -> - PlatformMetricsRegistry.registerCounter( - SPAN_STORE_COUNT, Map.of("tenantId", k))) - .increment(spanStore.approximateNumEntries() * 1.0); - } + recordSpansPerTrace(rawSpanList.size(), List.of(Tag.of("tenant_id", tenantId))); + Timestamps timestamps = + trackEndToEndLatencyTimestamps( + System.currentTimeMillis(), traceState.getTraceStartTimestamp()); + StructuredTrace trace = + StructuredTraceBuilder.buildStructuredTraceFromRawSpans( + rawSpanList, traceId, tenantId, timestamps); + + if (logger.isDebugEnabled()) { + logger.debug( + "Emit tenant_id=[{}], trace_id=[{}], spans_count=[{}]", + tenantId, + HexUtils.getHex(traceId), + rawSpanList.size()); + } - // report count of spanIds per trace - tenantToSpanPerTraceCounter + // report entries in spanStore + if (spanStoreCountRateLimiter.tryAcquire()) { + tenantToSpanStoreCountCounter .computeIfAbsent( tenantId, - k -> PlatformMetricsRegistry.registerCounter(SPANS_PER_TRACE, Map.of("tenantId", k))) - .increment(spanIds.size() * 1.0); + k -> PlatformMetricsRegistry.registerCounter(SPAN_STORE_COUNT, Map.of("tenantId", k))) + .increment(spanStore.approximateNumEntries() * 1.0); + } - // report trace emitted count - tenantToTraceEmittedCounter - .computeIfAbsent( - tenantId, - k -> - PlatformMetricsRegistry.registerCounter( - TRACES_EMITTER_COUNTER, Map.of("tenantId", k))) - .increment(); + // report count of spanIds per trace + tenantToSpanPerTraceCounter + .computeIfAbsent( + tenantId, + k -> PlatformMetricsRegistry.registerCounter(SPANS_PER_TRACE, Map.of("tenantId", k))) + .increment(spanIds.size() * 1.0); - // report punctuate latency - tenantToPunctuateLatencyTimer - .computeIfAbsent( - tenantId, - k -> - PlatformMetricsRegistry.registerTimer( - PUNCTUATE_LATENCY_TIMER, Map.of("tenantId", k))) - .record(Duration.between(startTime, Instant.now()).toMillis(), TimeUnit.MILLISECONDS); + // report trace emitted count + tenantToTraceEmittedCounter + .computeIfAbsent( + tenantId, + k -> + PlatformMetricsRegistry.registerCounter( + TRACES_EMITTER_COUNTER, Map.of("tenantId", k))) + .increment(); - context.forward(key, trace, outputTopicProducer); - } else { - // implies spans for the trace have arrived within the last 'sessionTimeoutMs' interval - // so the session inactivity window is extended from the last timestamp - if (logger.isDebugEnabled()) { - logger.debug( - "Re-scheduling emit trigger for tenant_id=[{}], trace_id=[{}] to [{}]", - key.getTenantId(), - HexUtils.getHex(key.getTraceId()), - Instant.ofEpochMilli(emitTs + groupingWindowTimeoutMs)); - } - long newEmitTs = emitTs + groupingWindowTimeoutMs; - // if current timestamp is ahead of newEmitTs then just add a grace of 100ms and fire it - long duration = Math.max(100, newEmitTs - timestamp); - cancellable = - context.schedule(Duration.ofMillis(duration), PunctuationType.WALL_CLOCK_TIME, this); - } + context.forward(key, trace, outputTopicProducer); } private Timestamps trackEndToEndLatencyTimestamps( 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 c70747fba..db470a961 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 @@ -61,3 +61,12 @@ metrics.reporter { } dataflow.metriccollection.sampling.percent = 10.0 + +trace.emit.callback.registry { + frequency = 1s + frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} + yield = 5s + yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} + window = 5ms + window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} +} \ No newline at end of file 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..d6cef19d8 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 @@ -205,12 +205,12 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir td.advanceWallClockTime(Duration.ofSeconds(1)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2); - // select a value < 30s (groupingWindowTimeoutInMs) - // this shouldn't trigger a punctuate call + // select a value < 5s (test config for groupingWindowTimeoutInMs) + // this shouldn't trigger a span emit td.advanceWallClockTime(Duration.ofMillis(200)); assertTrue(outputTopic.isEmpty()); - // the next advance should trigger a punctuate call and emit a trace with 2 spans + // the next advance should and emit a trace with 2 spans td.advanceWallClockTime(Duration.ofSeconds(32)); // trace1 should have 2 span span1, span2 @@ -231,7 +231,7 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span3); td.advanceWallClockTime(Duration.ofSeconds(45)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span5); - // the next advance should trigger a punctuate call and emit a trace with 2 spans + // the next advance should emit a trace with 2 spans td.advanceWallClockTime(Duration.ofSeconds(35)); // trace1 should have 1 span i.e. span3 diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java new file mode 100644 index 000000000..b3d5d43a5 --- /dev/null +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java @@ -0,0 +1,103 @@ +package org.hypertrace.core.rawspansgrouper; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.state.KeyValueStore; +import org.hypertrace.core.datamodel.Event; +import org.hypertrace.core.datamodel.RawSpan; +import org.hypertrace.core.kafkastreams.framework.callbacks.action.CallbackAction; +import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; +import org.hypertrace.core.spannormalizer.SpanIdentity; +import org.hypertrace.core.spannormalizer.TraceIdentity; +import org.hypertrace.core.spannormalizer.TraceState; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TraceEmitCallbackTest { + private static final long groupingWindowTimeoutMs = 300; + private static final TraceIdentity traceIdentity = + TraceIdentity.newBuilder() + .setTenantId("__default") + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .build(); + private TraceEmitCallback emitCallback; + private KeyValueStore spanStore; + private KeyValueStore traceStateStore; + + @BeforeEach + public void setUp() { + AvroSerde avroSerde = new AvroSerde(); + ProcessorContext context = mock(ProcessorContext.class); + when(context.keySerde()).thenReturn(avroSerde); + spanStore = mock(KeyValueStore.class); + traceStateStore = mock(KeyValueStore.class); + To outputTopicProducer = mock(To.class); + emitCallback = + new TraceEmitCallback( + context, spanStore, traceStateStore, outputTopicProducer, groupingWindowTimeoutMs, -1); + } + + @Test + public void testWhenWindowIsExtended() { + TraceState traceState = + TraceState.newBuilder() + .setSpanIds(List.of(ByteBuffer.wrap("span-1".getBytes()))) + .setEmitTs(-1) + .setTraceStartTimestamp(100) + .setTraceEndTimestamp(200) + .setTenantId("tenant") + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .build(); + when(traceStateStore.get(eq(traceIdentity))).thenReturn(traceState); + + CallbackAction callbackAction = emitCallback.callback(300, traceIdentity); + assertEquals( + traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs, + callbackAction.getRescheduleTimestamp().get()); + // the above when() call should be the only interaction + verify(traceStateStore, times(1)).get(traceIdentity); + } + + @Test + public void testWhenTraceToBeEmitted() { + TraceState traceState = + TraceState.newBuilder() + .setSpanIds(List.of(ByteBuffer.wrap("span-1".getBytes()))) + .setEmitTs(-1) + .setTraceStartTimestamp(100) + .setTraceEndTimestamp(130) + .setTenantId("tenant") + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .build(); + when(traceStateStore.get(eq(traceIdentity))).thenReturn(traceState); + + RawSpan rawSpan = + RawSpan.newBuilder() + .setCustomerId("__default") + .setEvent( + Event.newBuilder() + .setEventId(ByteBuffer.wrap("span-1".getBytes())) + .setCustomerId("__default") + .build()) + .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) + .build(); + when(spanStore.delete(any())).thenReturn(rawSpan); + + assertTrue(emitCallback.callback(450, traceIdentity).getRescheduleTimestamp().isEmpty()); + + verify(traceStateStore, times(1)).get(traceIdentity); + verify(spanStore, times(1)).delete(any()); + verify(traceStateStore).delete(eq(traceIdentity)); + } +} diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java deleted file mode 100644 index 43b526c42..000000000 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java +++ /dev/null @@ -1,117 +0,0 @@ -package org.hypertrace.core.rawspansgrouper; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.nio.ByteBuffer; -import java.util.List; -import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; -import org.apache.kafka.streams.state.KeyValueStore; -import org.hypertrace.core.datamodel.Event; -import org.hypertrace.core.datamodel.RawSpan; -import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; -import org.hypertrace.core.spannormalizer.SpanIdentity; -import org.hypertrace.core.spannormalizer.TraceIdentity; -import org.hypertrace.core.spannormalizer.TraceState; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -class TraceEmitPunctuatorTest { - - private TraceEmitPunctuator underTest; - private KeyValueStore spanStore; - private KeyValueStore traceStateStore; - - @BeforeEach - public void setUp() { - AvroSerde avroSerde = new AvroSerde(); - ProcessorContext context = mock(ProcessorContext.class); - when(context.keySerde()).thenReturn(avroSerde); - spanStore = mock(KeyValueStore.class); - traceStateStore = mock(KeyValueStore.class); - To outputTopicProducer = mock(To.class); - underTest = - new TraceEmitPunctuator( - TraceIdentity.newBuilder() - .setTenantId("__default") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build(), - context, - spanStore, - traceStateStore, - outputTopicProducer, - 100, - -1); - underTest.setCancellable(mock(Cancellable.class)); - } - - @Test - public void whenPunctuatorIsRescheduledExpectNoChangesToTraceEmitTriggerStore() { - when(traceStateStore.get( - eq( - TraceIdentity.newBuilder() - .setTenantId("__default") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build()))) - .thenReturn( - TraceState.newBuilder() - .setSpanIds(List.of(ByteBuffer.wrap("span-1".getBytes()))) - .setEmitTs(300) - .setTraceStartTimestamp(150) - .setTraceEndTimestamp(300) - .setTenantId("tenant") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build()); - underTest.punctuate(200); - // the above when() call should be the only interaction - verify(traceStateStore, times(1)).get(any()); - } - - @Test - public void whenTraceIsEmittedExpectDeleteOperationOnTraceStateStore() { - - when(traceStateStore.get( - eq( - TraceIdentity.newBuilder() - .setTenantId("__default") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build()))) - .thenReturn( - TraceState.newBuilder() - .setSpanIds(List.of(ByteBuffer.wrap("span-1".getBytes()))) - .setEmitTs(180) - .setTraceStartTimestamp(100) - .setTraceEndTimestamp(130) - .setTenantId("tenant") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build()); - - when(spanStore.delete(any())) - .thenReturn( - RawSpan.newBuilder() - .setCustomerId("__default") - .setEvent( - Event.newBuilder() - .setEventId(ByteBuffer.wrap("span-1".getBytes())) - .setCustomerId("__default") - .build()) - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .build()); - underTest.punctuate(200); // the above when() call should be the only interaction - verify(traceStateStore, times(1)).get(any()); - verify(spanStore, times(1)).delete(any()); - verify(traceStateStore) - .delete( - eq( - TraceIdentity.newBuilder() - .setTenantId("__default") - .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) - .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 5284b3714..d889179e1 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,9 @@ span.groupby.session.window.interval = 5 span.groupby.session.window.graceperiod.ms = 100 dataflow.metriccollection.sampling.percent = 10.0 + +trace.emit.callback.registry { + frequency = 1s + yield = 5s + window = 5ms +} \ No newline at end of file diff --git a/span-normalizer/span-normalizer-api/src/main/avro/TraceState.avdl b/span-normalizer/span-normalizer-api/src/main/avro/TraceState.avdl index f651a183d..49520ca30 100644 --- a/span-normalizer/span-normalizer-api/src/main/avro/TraceState.avdl +++ b/span-normalizer/span-normalizer-api/src/main/avro/TraceState.avdl @@ -3,7 +3,7 @@ protocol TraceStateProtocol { record TraceState { long trace_start_timestamp; long trace_end_timestamp; - long emit_ts; + /** @deprecated unused */ long emit_ts; string tenant_id; bytes trace_id; array span_ids; From e5d10b220335ced19e66dd7107ad8d02f804876d Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Wed, 4 Oct 2023 16:53:07 +0530 Subject: [PATCH 02/31] helmify --- .../helm/templates/raw-spans-grouper-config.yaml | 8 ++++++++ raw-spans-grouper/helm/values.yaml | 4 ++++ 2 files changed, 12 insertions(+) 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 220bacf1c..eb86ba4bc 100644 --- a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml +++ b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml @@ -81,3 +81,11 @@ data: } } {{- end }} + + {{- if hasKey .Values.rawSpansGrouperConfig "traceEmitCallbackRegistry" }} + trace.emit.callback.registry { + frequency = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.frequency }} + yield = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.yield }} + window = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.window }} + } + {{- end }} diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 982468959..c18d52b51 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -132,6 +132,10 @@ rawSpansGrouperConfig: groupPartitionerEnabled: false groupPartitionerConfigServiceHost: config-service groupPartitionerConfigServicePort: 50104 + traceEmitCallbackRegistry: + frequency: 1s + yield: 10s + window: 5ms span: groupby: From 94d49252ccc922f8468c3a6a375f8217dd29358c Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Wed, 4 Oct 2023 16:56:49 +0530 Subject: [PATCH 03/31] nit --- .../hypertrace/core/rawspansgrouper/RawSpansGrouperTest.java | 2 +- .../test/resources/configs/raw-spans-grouper/application.conf | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) 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 d6cef19d8..d70fb5540 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 @@ -205,7 +205,7 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir td.advanceWallClockTime(Duration.ofSeconds(1)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2); - // select a value < 5s (test config for groupingWindowTimeoutInMs) + // select a value 30s (groupingWindowTimeoutInMs) // this shouldn't trigger a span emit td.advanceWallClockTime(Duration.ofMillis(200)); assertTrue(outputTopic.isEmpty()); 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 d889179e1..e250a3616 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 @@ -39,14 +39,14 @@ max.span.count = { tenant1 = 5 } -span.groupby.session.window.interval = 5 +span.groupby.session.window.interval = 30 span.groupby.session.window.graceperiod.ms = 100 dataflow.metriccollection.sampling.percent = 10.0 trace.emit.callback.registry { - frequency = 1s + frequency = 15s yield = 5s window = 5ms } \ No newline at end of file From 59eaa47df3426199c56d5ab8ab4baba4dc29fa0d Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Thu, 5 Oct 2023 12:47:38 +0530 Subject: [PATCH 04/31] refactor --- .../rawspansgrouper/RawSpansProcessor.java | 32 ++++++++----------- ...ck.java => TraceEmitCallbackRegistry.java} | 19 +++++------ ...ava => TraceEmitCallbackRegistryTest.java} | 16 +++++++--- 3 files changed, 36 insertions(+), 31 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{TraceEmitCallback.java => TraceEmitCallbackRegistry.java} (92%) rename raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/{TraceEmitCallbackTest.java => TraceEmitCallbackRegistryTest.java} (89%) 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 f1dd85fd7..bbaa99a2f 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 @@ -39,7 +39,6 @@ import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.shared.HexUtils; -import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuator; import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; @@ -49,11 +48,11 @@ import org.slf4j.LoggerFactory; /** - * Receives spans keyed by trace_id and stores them. A {@link TraceEmitCallback} is scheduled to run - * after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If any - * spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval - * then the trace will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} time to - * accept spans. + * Receives spans keyed by trace_id and stores them. A {@link TraceEmitCallbackRegistry} is + * scheduled to run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the + * trace. If any spans for the trace arrive within the {@link + * RawSpansProcessor#groupingWindowTimeoutMs} interval then the trace will get an additional {@link + * RawSpansProcessor#groupingWindowTimeoutMs} time to accept spans. */ public class RawSpansProcessor implements Transformer> { @@ -78,7 +77,7 @@ public class RawSpansProcessor private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; - private CallbackRegistryPunctuator traceEmitCallbackRegistry; + private TraceEmitCallbackRegistry traceEmitCallbackRegistry; private Cancellable traceEmitCallbackRegistryCancellable; @Override @@ -113,23 +112,20 @@ public void init(ProcessorContext context) { this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); - TraceEmitCallback traceEmitCallback = - new TraceEmitCallback( - context, - spanStore, - traceStateStore, - outputTopic, - groupingWindowTimeoutMs, - dataflowSamplingPercent); KeyValueStore> traceEmitCallbackRegistryStore = context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); traceEmitCallbackRegistry = - new CallbackRegistryPunctuator<>( + new TraceEmitCallbackRegistry( new CallbackRegistryPunctuatorConfig( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY).toMillis(), jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY).toMillis()), traceEmitCallbackRegistryStore, - traceEmitCallback::callback); + context, + spanStore, + traceStateStore, + outputTopic, + groupingWindowTimeoutMs, + dataflowSamplingPercent); traceEmitCallbackRegistryCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), @@ -163,7 +159,7 @@ public KeyValue transform(TraceIdentity key, Raw .setTraceId(traceId) .setSpanIds(List.of(spanId)) .build(); - traceEmitCallbackRegistry.add(currentTimeMs, key); + traceEmitCallbackRegistry.invoke(currentTimeMs, key); } else { traceState.getSpanIds().add(spanId); traceState.setTraceEndTimestamp(currentTimeMs); diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallback.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java similarity index 92% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallback.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java index 959f47ed9..b639942a6 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallback.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java @@ -28,6 +28,8 @@ 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.kafkastreams.framework.callbacks.AbstractCallbackRegistryPunctuator; +import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; import org.hypertrace.core.kafkastreams.framework.callbacks.action.CallbackAction; import org.hypertrace.core.kafkastreams.framework.callbacks.action.DropCallbackAction; import org.hypertrace.core.kafkastreams.framework.callbacks.action.RescheduleCallbackAction; @@ -39,12 +41,12 @@ import org.slf4j.LoggerFactory; /** - * Callback to check if a trace can be finalized and emitted based on inactivity period of {@link + * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansProcessor#groupingWindowTimeoutMs} */ -class TraceEmitCallback { +class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator { - private static final Logger logger = LoggerFactory.getLogger(TraceEmitCallback.class); + private static final Logger logger = LoggerFactory.getLogger(TraceEmitCallbackRegistry.class); private static final Object mutex = new Object(); private static final Timer spansGrouperArrivalLagTimer = @@ -52,10 +54,6 @@ class TraceEmitCallback { private static final String TRACES_EMITTER_COUNTER = "hypertrace.emitted.traces"; private static final ConcurrentMap tenantToTraceEmittedCounter = new ConcurrentHashMap<>(); - private static final String PUNCTUATE_LATENCY_TIMER = - "hypertrace.rawspansgrouper.punctuate.latency"; - private static final ConcurrentMap tenantToPunctuateLatencyTimer = - new ConcurrentHashMap<>(); private static final String SPANS_PER_TRACE = "hypertrace.rawspansgrouper.spans.per.trace"; private static final ConcurrentMap tenantToSpanPerTraceCounter = new ConcurrentHashMap<>(); @@ -76,13 +74,16 @@ class TraceEmitCallback { private final To outputTopicProducer; private final long groupingWindowTimeoutMs; - TraceEmitCallback( + TraceEmitCallbackRegistry( + CallbackRegistryPunctuatorConfig callbackRegistryPunctuatorConfig, + KeyValueStore> callbackRegistryStore, ProcessorContext context, KeyValueStore spanStore, KeyValueStore traceStateStore, To outputTopicProducer, long groupingWindowTimeoutMs, double dataflowSamplingPercent) { + super(callbackRegistryPunctuatorConfig, callbackRegistryStore); this.context = context; this.spanStore = spanStore; this.traceStateStore = traceStateStore; @@ -91,7 +92,7 @@ class TraceEmitCallback { this.dataflowSamplingPercent = dataflowSamplingPercent; } - CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { + protected CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { TraceState traceState = traceStateStore.get(key); if (null == traceState || null == traceState.getSpanIds() diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java similarity index 89% rename from raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java rename to raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java index b3d5d43a5..865280df6 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java @@ -16,6 +16,7 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; +import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; import org.hypertrace.core.kafkastreams.framework.callbacks.action.CallbackAction; import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; import org.hypertrace.core.spannormalizer.SpanIdentity; @@ -24,14 +25,14 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -class TraceEmitCallbackTest { +class TraceEmitCallbackRegistryTest { private static final long groupingWindowTimeoutMs = 300; private static final TraceIdentity traceIdentity = TraceIdentity.newBuilder() .setTenantId("__default") .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) .build(); - private TraceEmitCallback emitCallback; + private TraceEmitCallbackRegistry emitCallback; private KeyValueStore spanStore; private KeyValueStore traceStateStore; @@ -44,8 +45,15 @@ public void setUp() { traceStateStore = mock(KeyValueStore.class); To outputTopicProducer = mock(To.class); emitCallback = - new TraceEmitCallback( - context, spanStore, traceStateStore, outputTopicProducer, groupingWindowTimeoutMs, -1); + new TraceEmitCallbackRegistry( + mock(CallbackRegistryPunctuatorConfig.class), + mock(KeyValueStore.class), + context, + spanStore, + traceStateStore, + outputTopicProducer, + groupingWindowTimeoutMs, + -1); } @Test From 6cdc527e2bcc760f8f4d5644e49fda40d6b5e9e3 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Thu, 5 Oct 2023 19:24:26 +0530 Subject: [PATCH 05/31] nit --- .../RawSpanGrouperConstants.java | 2 +- .../core/rawspansgrouper/RawSpansProcessor.java | 14 ++++++-------- .../TraceEmitCallbackRegistry.java | 14 ++++++++------ .../resources/configs/common/application.conf | 17 ++++++++++++----- 4 files changed, 27 insertions(+), 20 deletions(-) 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 f89672d21..ab040257e 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,7 +18,7 @@ public class RawSpanGrouperConstants { public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; public static final String TRUNCATED_TRACES_COUNTER = "hypertrace.truncated.traces"; static final String TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME = - "trace-emit-callback-registry-store"; + "test-kishan-trace-emit-callback-registry-store"; // TODO: remove test after done with it static final String TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY = "trace.emit.callback.registry.frequency"; static final String TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY = 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 bbaa99a2f..b9ce81277 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 @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.time.Duration; import java.time.Instant; +import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -69,7 +70,6 @@ public class RawSpansProcessor // counter for number of truncated traces per tenant private static final ConcurrentMap truncatedTracesCounter = new ConcurrentHashMap<>(); - private ProcessorContext context; private KeyValueStore spanStore; private KeyValueStore traceStateStore; private long groupingWindowTimeoutMs; @@ -82,11 +82,8 @@ public class RawSpansProcessor @Override public void init(ProcessorContext context) { - this.context = context; - this.spanStore = - (KeyValueStore) context.getStateStore(SPAN_STATE_STORE_NAME); - this.traceStateStore = - (KeyValueStore) context.getStateStore(TRACE_STATE_STORE); + this.spanStore = context.getStateStore(SPAN_STATE_STORE_NAME); + this.traceStateStore = context.getStateStore(TRACE_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); this.groupingWindowTimeoutMs = jobConfig.getLong(SPAN_GROUPBY_SESSION_WINDOW_INTERVAL_CONFIG_KEY) * 1000; @@ -99,7 +96,8 @@ public void init(ProcessorContext context) { 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())); @@ -112,7 +110,7 @@ public void init(ProcessorContext context) { this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); - KeyValueStore> traceEmitCallbackRegistryStore = + KeyValueStore> traceEmitCallbackRegistryStore = context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); traceEmitCallbackRegistry = new TraceEmitCallbackRegistry( diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java index b639942a6..f09f53123 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java @@ -9,6 +9,7 @@ import io.micrometer.core.instrument.Tag; import io.micrometer.core.instrument.Timer; import java.nio.ByteBuffer; +import java.time.Clock; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -76,14 +77,14 @@ class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator> callbackRegistryStore, + KeyValueStore> callbackRegistryStore, ProcessorContext context, KeyValueStore spanStore, KeyValueStore traceStateStore, To outputTopicProducer, long groupingWindowTimeoutMs, double dataflowSamplingPercent) { - super(callbackRegistryPunctuatorConfig, callbackRegistryStore); + super(Clock.systemUTC(), callbackRegistryPunctuatorConfig, callbackRegistryStore); this.context = context; this.spanStore = spanStore; this.traceStateStore = traceStateStore; @@ -97,10 +98,11 @@ protected CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { if (null == traceState || null == traceState.getSpanIds() || traceState.getSpanIds().isEmpty()) { - logger.warn( - "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", - key.getTenantId(), - HexUtils.getHex(key.getTraceId())); + // TODO: uncomment, commented for debugging purpose + // logger.warn( + // "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", + // key.getTenantId(), + // HexUtils.getHex(key.getTraceId())); return DROP_CALLBACK_ACTION; } if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { 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 db470a961..cf45ae1e2 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 @@ -62,11 +62,18 @@ metrics.reporter { dataflow.metriccollection.sampling.percent = 10.0 +// TODO: revert after tuning +//trace.emit.callback.registry { +// frequency = 30s +// frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} +// yield = 30s +// yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} +// window = 5ms +// window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} +//} + trace.emit.callback.registry { - frequency = 1s - frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} - yield = 5s - yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} + frequency = 10ms + yield = 10ms window = 5ms - window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} } \ No newline at end of file From 354e9c9d32ceb47ac543a538b1c7e8f0daaec0be Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Thu, 5 Oct 2023 20:16:58 +0530 Subject: [PATCH 06/31] tune --- .../TraceEmitCallbackRegistry.java | 9 ++++----- .../resources/configs/common/application.conf | 15 ++++----------- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java index f09f53123..8ae2da028 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java @@ -98,11 +98,10 @@ protected CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { if (null == traceState || null == traceState.getSpanIds() || traceState.getSpanIds().isEmpty()) { - // TODO: uncomment, commented for debugging purpose - // logger.warn( - // "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", - // key.getTenantId(), - // HexUtils.getHex(key.getTraceId())); + logger.warn( + "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", + key.getTenantId(), + HexUtils.getHex(key.getTraceId())); return DROP_CALLBACK_ACTION; } if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { 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 cf45ae1e2..1ca6beef7 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 @@ -62,18 +62,11 @@ metrics.reporter { dataflow.metriccollection.sampling.percent = 10.0 -// TODO: revert after tuning -//trace.emit.callback.registry { -// frequency = 30s -// frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} -// yield = 30s -// yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} -// window = 5ms -// window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} -//} - trace.emit.callback.registry { - frequency = 10ms + frequency = 100ms + frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} yield = 10ms + yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} window = 5ms + window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} } \ No newline at end of file From cfb9819236b91fd82a8711187228d355e1813080 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 6 Oct 2023 20:35:37 +0530 Subject: [PATCH 07/31] edit --- .../core/rawspansgrouper/RawSpansProcessor.java | 2 +- .../core/rawspansgrouper/TraceEmitCallbackRegistry.java | 8 ++++---- .../src/main/resources/configs/common/application.conf | 7 +++---- 3 files changed, 8 insertions(+), 9 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 b9ce81277..be83f12de 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 @@ -127,7 +127,7 @@ public void init(ProcessorContext context) { traceEmitCallbackRegistryCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), - PunctuationType.WALL_CLOCK_TIME, + PunctuationType.STREAM_TIME, traceEmitCallbackRegistry); } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java index 8ae2da028..18d960661 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java @@ -98,10 +98,10 @@ protected CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { if (null == traceState || null == traceState.getSpanIds() || traceState.getSpanIds().isEmpty()) { - logger.warn( - "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", - key.getTenantId(), - HexUtils.getHex(key.getTraceId())); +// logger.warn( +// "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", +// key.getTenantId(), +// HexUtils.getHex(key.getTraceId())); return DROP_CALLBACK_ACTION; } if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { 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 1ca6beef7..3d6baa6e0 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 @@ -29,7 +29,6 @@ kafka.streams.config = { schema.registry.url = "http://localhost:8081" schema.registry.url = ${?SCHEMA_REGISTRY_URL} - rocksdb.config.setter = org.hypertrace.core.kafkastreams.framework.rocksdb.RocksDBStateStoreConfigSetter rocksdb.block.cache.size = 33554432 rocksdb.write.buffer.size = 8388608 rocksdb.max.write.buffers = 2 @@ -63,10 +62,10 @@ metrics.reporter { dataflow.metriccollection.sampling.percent = 10.0 trace.emit.callback.registry { - frequency = 100ms + frequency = 30s frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} - yield = 10ms + yield = 15s yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} - window = 5ms + window = 1ms window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} } \ No newline at end of file From 26dff952cabd80490c088adbf91099f65a9a47ee Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Mon, 9 Oct 2023 13:39:05 +0530 Subject: [PATCH 08/31] wrap up for wall clock time changes --- .../rawspansgrouper/RawSpansProcessor.java | 13 ++++--- .../TraceEmitCallbackRegistry.java | 35 +++++++++---------- .../rawspansgrouper/RawSpansGrouperTest.java | 16 ++++----- .../TraceEmitCallbackRegistryTest.java | 8 ++--- 4 files changed, 38 insertions(+), 34 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 be83f12de..de58f960d 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 @@ -40,7 +40,7 @@ import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.datamodel.shared.HexUtils; -import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; +import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -114,7 +114,7 @@ public void init(ProcessorContext context) { context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); traceEmitCallbackRegistry = new TraceEmitCallbackRegistry( - new CallbackRegistryPunctuatorConfig( + new ThrottledPunctuatorConfig( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY).toMillis(), jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY).toMillis()), traceEmitCallbackRegistryStore, @@ -127,7 +127,7 @@ public void init(ProcessorContext context) { traceEmitCallbackRegistryCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), - PunctuationType.STREAM_TIME, + PunctuationType.WALL_CLOCK_TIME, traceEmitCallbackRegistry); } @@ -157,10 +157,15 @@ public KeyValue transform(TraceIdentity key, Raw .setTraceId(traceId) .setSpanIds(List.of(spanId)) .build(); - traceEmitCallbackRegistry.invoke(currentTimeMs, key); + traceEmitCallbackRegistry.scheduleTask(currentTimeMs, key); } else { traceState.getSpanIds().add(spanId); + long prevScheduleTimestamp = traceState.getTraceEndTimestamp(); traceState.setTraceEndTimestamp(currentTimeMs); + if(!traceEmitCallbackRegistry.rescheduleTask( + prevScheduleTimestamp, currentTimeMs + groupingWindowTimeoutMs, key)) { + logger.debug("Failed to reschedule task for trace key {}, schedule already dropped!", key); + } } traceStateStore.put(key, traceState); diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java index 18d960661..e6eb9813c 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java @@ -29,11 +29,11 @@ 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.kafkastreams.framework.callbacks.AbstractCallbackRegistryPunctuator; -import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.callbacks.action.CallbackAction; -import org.hypertrace.core.kafkastreams.framework.callbacks.action.DropCallbackAction; -import org.hypertrace.core.kafkastreams.framework.callbacks.action.RescheduleCallbackAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.AbstractThrottledPunctuator; +import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.DropScheduleAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.RescheduleAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.ScheduleAction; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -45,7 +45,7 @@ * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansProcessor#groupingWindowTimeoutMs} */ -class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator { +class TraceEmitCallbackRegistry extends AbstractThrottledPunctuator { private static final Logger logger = LoggerFactory.getLogger(TraceEmitCallbackRegistry.class); private static final Object mutex = new Object(); @@ -67,7 +67,7 @@ class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); - private static final DropCallbackAction DROP_CALLBACK_ACTION = new DropCallbackAction(); + private static final DropScheduleAction DROP_SCHEDULE_ACTION = new DropScheduleAction(); private final double dataflowSamplingPercent; private final ProcessorContext context; private final KeyValueStore spanStore; @@ -76,7 +76,7 @@ class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator> callbackRegistryStore, ProcessorContext context, KeyValueStore spanStore, @@ -84,7 +84,7 @@ class TraceEmitCallbackRegistry extends AbstractCallbackRegistryPunctuator= groupingWindowTimeoutMs) { // Implies that no new spans for the trace have arrived within the last @@ -110,10 +110,9 @@ protected CallbackAction callback(long punctuateTimestamp, TraceIdentity key) { // so the trace can be finalized and emitted emitTrace(key, traceState); // no need of running again for this - return DROP_CALLBACK_ACTION; + return DROP_SCHEDULE_ACTION; } - return new RescheduleCallbackAction( - traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); + return new RescheduleAction(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); } private void emitTrace(TraceIdentity key, TraceState traceState) { 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 d70fb5540..00b06dcf2 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 @@ -205,7 +205,7 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir td.advanceWallClockTime(Duration.ofSeconds(1)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2); - // select a value 30s (groupingWindowTimeoutInMs) + // select a value < 30s (groupingWindowTimeoutInMs) // this shouldn't trigger a span emit td.advanceWallClockTime(Duration.ofMillis(200)); assertTrue(outputTopic.isEmpty()); @@ -213,8 +213,13 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir // the next advance should and emit a trace with 2 spans td.advanceWallClockTime(Duration.ofSeconds(32)); - // trace1 should have 2 span span1, span2 + // trace2 should have 1 span span3 StructuredTrace trace = outputTopic.readValue(); + assertEquals(1, trace.getEventList().size()); + assertEquals("event-4", new String(trace.getEventList().get(0).getEventId().array())); + + // trace1 should have 2 span span1, span2 + trace = outputTopic.readValue(); assertEquals(2, trace.getEventList().size()); Set traceEventIds = trace.getEventList().stream() @@ -223,11 +228,6 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertTrue(traceEventIds.contains("event-1")); assertTrue(traceEventIds.contains("event-2")); - // trace2 should have 1 span span3 - trace = outputTopic.readValue(); - assertEquals(1, trace.getEventList().size()); - assertEquals("event-4", new String(trace.getEventList().get(0).getEventId().array())); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span3); td.advanceWallClockTime(Duration.ofSeconds(45)); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span5); @@ -252,7 +252,7 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span11); td.advanceWallClockTime(Duration.ofSeconds(35)); - // trace should be truncated with 5 spans + // trace3 should be truncated with 5 spans trace = outputTopic.readValue(); assertEquals(5, trace.getEventList().size()); diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java index 865280df6..b38c95821 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java @@ -16,8 +16,8 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; -import org.hypertrace.core.kafkastreams.framework.callbacks.CallbackRegistryPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.callbacks.action.CallbackAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.ScheduleAction; import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -46,7 +46,7 @@ public void setUp() { To outputTopicProducer = mock(To.class); emitCallback = new TraceEmitCallbackRegistry( - mock(CallbackRegistryPunctuatorConfig.class), + mock(ThrottledPunctuatorConfig.class), mock(KeyValueStore.class), context, spanStore, @@ -69,7 +69,7 @@ public void testWhenWindowIsExtended() { .build(); when(traceStateStore.get(eq(traceIdentity))).thenReturn(traceState); - CallbackAction callbackAction = emitCallback.callback(300, traceIdentity); + ScheduleAction callbackAction = emitCallback.callback(300, traceIdentity); assertEquals( traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs, callbackAction.getRescheduleTimestamp().get()); From 94f957acb6ed40d37ace405a669348d46b67bcaa Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Mon, 9 Oct 2023 14:37:24 +0530 Subject: [PATCH 09/31] try changing clock --- .../core/rawspansgrouper/RawSpansGrouper.java | 14 +++- .../rawspansgrouper/RawSpansProcessor.java | 43 +++++++--- ...try.java => TraceEmitTasksPunctuator.java} | 6 +- .../rawspansgrouper/RawSpansGrouperTest.java | 84 +++++++++++-------- ...java => TraceEmitTasksPunctuatorTest.java} | 6 +- 5 files changed, 100 insertions(+), 53 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{TraceEmitCallbackRegistry.java => TraceEmitTasksPunctuator.java} (98%) rename raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/{TraceEmitCallbackRegistryTest.java => TraceEmitTasksPunctuatorTest.java} (97%) 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 85116b14e..68f1810f5 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 @@ -8,7 +8,10 @@ import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_STATE_STORE; +import com.google.common.annotations.VisibleForTesting; import com.typesafe.config.Config; + +import java.time.Clock; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -44,6 +47,15 @@ public RawSpansGrouper(ConfigClient configClient) { } public StreamsBuilder buildTopology( + Map properties, + StreamsBuilder streamsBuilder, + Map> inputStreams) { + return buildTopologyWithClock(Clock.systemUTC(), properties, streamsBuilder, inputStreams); + } + + @VisibleForTesting + StreamsBuilder buildTopologyWithClock( + Clock clock, Map properties, StreamsBuilder streamsBuilder, Map> inputStreams) { @@ -101,7 +113,7 @@ public StreamsBuilder buildTopology( inputStream .transform( - RawSpansProcessor::new, + () -> new RawSpansProcessor(clock), Named.as(RawSpansProcessor.class.getSimpleName()), SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, 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 de58f960d..6ecc5f2f7 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 @@ -20,6 +20,7 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.Timer; import java.nio.ByteBuffer; +import java.time.Clock; import java.time.Duration; import java.time.Instant; import java.util.ArrayList; @@ -49,7 +50,7 @@ import org.slf4j.LoggerFactory; /** - * Receives spans keyed by trace_id and stores them. A {@link TraceEmitCallbackRegistry} is + * Receives spans keyed by trace_id and stores them. A {@link TraceEmitTasksPunctuator} is * scheduled to run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the * trace. If any spans for the trace arrive within the {@link * RawSpansProcessor#groupingWindowTimeoutMs} interval then the trace will get an additional {@link @@ -70,6 +71,7 @@ public class RawSpansProcessor // counter for number of truncated traces per tenant private static final ConcurrentMap truncatedTracesCounter = new ConcurrentHashMap<>(); + private final Clock clock; private KeyValueStore spanStore; private KeyValueStore traceStateStore; private long groupingWindowTimeoutMs; @@ -77,8 +79,12 @@ public class RawSpansProcessor private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; - private TraceEmitCallbackRegistry traceEmitCallbackRegistry; - private Cancellable traceEmitCallbackRegistryCancellable; + private TraceEmitTasksPunctuator traceEmitTasksPunctuator; + private Cancellable traceEmitTasksPunctuatorCancellable; + + public RawSpansProcessor(Clock clock) { + this.clock = clock; + } @Override public void init(ProcessorContext context) { @@ -112,8 +118,8 @@ public void init(ProcessorContext context) { KeyValueStore> traceEmitCallbackRegistryStore = context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); - traceEmitCallbackRegistry = - new TraceEmitCallbackRegistry( + traceEmitTasksPunctuator = + new TraceEmitTasksPunctuator( new ThrottledPunctuatorConfig( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY).toMillis(), jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY).toMillis()), @@ -124,19 +130,30 @@ public void init(ProcessorContext context) { outputTopic, groupingWindowTimeoutMs, dataflowSamplingPercent); - traceEmitCallbackRegistryCancellable = + // Punctuator scheduled on stream time => no input messages => no emits will happen + // We will almost never have input down to 0, i.e., there are no spans coming to platform, + // While using wall clock time handles that case, there is a issue with using wall clock time... + // In cases of lag being burnt, we are processing message produced at different time stamp intervals + // probably faster than at rate which they were produced, now not doing punctuation often will increase the + // amounts of work for punctuator in next iterations and will keep on piling up until lag is burnt completely + // and only then the punctuator will catch up back to normal input rate. This is undesirable, here the outputs + // are only emitted from punctuator, if we burn lag from inputs, we want to push it down to downstream as soon + // as possible, if we hog it more and more it will delay cascading lag to downstream. Given grouper stays at start + // of pipeline it is better to use stream time as using wall clock time can have more undesirable effects + traceEmitTasksPunctuatorCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), - PunctuationType.WALL_CLOCK_TIME, - traceEmitCallbackRegistry); + PunctuationType.STREAM_TIME, + traceEmitTasksPunctuator); } public KeyValue transform(TraceIdentity key, RawSpan value) { Instant start = Instant.now(); - long currentTimeMs = System.currentTimeMillis(); + long currentTimeMs = clock.millis(); TraceState traceState = traceStateStore.get(key); boolean firstEntry = (traceState == null); + ByteBuffer debugSpanId = value.getEvent().getEventId(); if (shouldDropSpan(key, traceState)) { return null; @@ -157,14 +174,14 @@ public KeyValue transform(TraceIdentity key, Raw .setTraceId(traceId) .setSpanIds(List.of(spanId)) .build(); - traceEmitCallbackRegistry.scheduleTask(currentTimeMs, key); + traceEmitTasksPunctuator.scheduleTask(currentTimeMs, key); } else { traceState.getSpanIds().add(spanId); long prevScheduleTimestamp = traceState.getTraceEndTimestamp(); traceState.setTraceEndTimestamp(currentTimeMs); - if(!traceEmitCallbackRegistry.rescheduleTask( + if(!traceEmitTasksPunctuator.rescheduleTask( prevScheduleTimestamp, currentTimeMs + groupingWindowTimeoutMs, key)) { - logger.debug("Failed to reschedule task for trace key {}, schedule already dropped!", key); + logger.debug("Failed to reschedule task on getting span for trace key {}, schedule already dropped!", key); } } @@ -227,6 +244,6 @@ private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { @Override public void close() { - traceEmitCallbackRegistryCancellable.cancel(); + traceEmitTasksPunctuatorCancellable.cancel(); } } diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java similarity index 98% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java index e6eb9813c..e7226b3be 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistry.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java @@ -45,9 +45,9 @@ * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansProcessor#groupingWindowTimeoutMs} */ -class TraceEmitCallbackRegistry extends AbstractThrottledPunctuator { +class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator { - private static final Logger logger = LoggerFactory.getLogger(TraceEmitCallbackRegistry.class); + private static final Logger logger = LoggerFactory.getLogger(TraceEmitTasksPunctuator.class); private static final Object mutex = new Object(); private static final Timer spansGrouperArrivalLagTimer = @@ -75,7 +75,7 @@ class TraceEmitCallbackRegistry extends AbstractThrottledPunctuator> callbackRegistryStore, ProcessorContext context, 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 00b06dcf2..d4599b4e2 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 @@ -2,6 +2,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import com.typesafe.config.Config; import com.typesafe.config.ConfigFactory; @@ -9,7 +11,7 @@ import java.io.File; import java.nio.ByteBuffer; import java.nio.file.Path; -import java.time.Duration; +import java.time.Clock; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -52,8 +54,9 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir mergedProps.put(RawSpanGrouperConstants.RAW_SPANS_GROUPER_JOB_CONFIG, config); mergedProps.put(StreamsConfig.STATE_DIR_CONFIG, file.getAbsolutePath()); + Clock clock = mock(Clock.class); StreamsBuilder streamsBuilder = - underTest.buildTopology(mergedProps, new StreamsBuilder(), new HashMap<>()); + underTest.buildTopologyWithClock(clock, mergedProps, new StreamsBuilder(), new HashMap<>()); Properties props = new Properties(); mergedProps.forEach(props::put); @@ -148,6 +151,12 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir .setCustomerId("tenant1") .setEvent(createEvent("event-11", "tenant1")) .build(); + RawSpan span20 = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) + .setCustomerId("tenant1") + .setEvent(createEvent("event-20", "tenant1")) + .build(); // create 8 spans for tenant-2 for trace-4 String tenant2 = "tenant2"; @@ -200,18 +209,19 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir .setEvent(createEvent("event-19", tenant2)) .build(); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span4); - td.advanceWallClockTime(Duration.ofSeconds(1)); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2); + long messageTime = advanceAndSyncClockMock(0, clock, 0); + + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1, messageTime); + advanceAndSyncClockMock(messageTime, clock, 1000); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2, messageTime); // select a value < 30s (groupingWindowTimeoutInMs) // this shouldn't trigger a span emit - td.advanceWallClockTime(Duration.ofMillis(200)); assertTrue(outputTopic.isEmpty()); - // the next advance should and emit a trace with 2 spans - td.advanceWallClockTime(Duration.ofSeconds(32)); + // the next advance should and emit a trace1 with 2 spans, trace2 with one span + advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span4, messageTime); // trace2 should have 1 span span3 StructuredTrace trace = outputTopic.readValue(); @@ -228,11 +238,12 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertTrue(traceEventIds.contains("event-1")); assertTrue(traceEventIds.contains("event-2")); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span3); - td.advanceWallClockTime(Duration.ofSeconds(45)); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span5); - // the next advance should emit a trace with 2 spans - td.advanceWallClockTime(Duration.ofSeconds(35)); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span3, messageTime); + advanceAndSyncClockMock(messageTime, clock, 45_000); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span5, messageTime); + // the next advance should emit a trace1, trace2 with one span + advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span6, messageTime); // trace1 should have 1 span i.e. span3 trace = outputTopic.readValue(); @@ -244,29 +255,30 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertEquals(1, trace.getEventList().size()); assertEquals("event-5", new String(trace.getEventList().get(0).getEventId().array())); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span6); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span7); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span8); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span9); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span10); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span11); - td.advanceWallClockTime(Duration.ofSeconds(35)); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span7, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span8, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span9, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span10, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span11, messageTime); + // the next advance should emit trace3 + advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span20, messageTime); - // trace3 should be truncated with 5 spans + // trace3 should be truncated with 5 spans because of tenant limit trace = outputTopic.readValue(); assertEquals(5, trace.getEventList().size()); - // input 8 spans of trace-4 for tenant2, as there is global upper limit apply, it will emit only - // 6 - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span12); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span13); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span14); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span15); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span16); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span17); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span18); - inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span19); - td.advanceWallClockTime(Duration.ofSeconds(35)); + // input 8 spans of trace-4 for tenant2, as there is global upper limit apply, it will emit only 6 + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span12, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span13, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span14, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span15, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span16, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span17, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span18, messageTime); + // the next advance should emit trace 4 + advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span19, messageTime); TestRecord testRecord = outputTopic.readRecord(); @@ -274,6 +286,12 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertEquals(6, testRecord.getValue().getEventList().size()); } + private long advanceAndSyncClockMock(long messageTime, Clock clock, long advanceMs) { + long finalMessageTime = messageTime + advanceMs; + when(clock.millis()).thenAnswer((inv) -> finalMessageTime); + return finalMessageTime; + } + private Event createEvent(String eventId, String tenantId) { return Event.newBuilder() .setCustomerId(tenantId) diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java similarity index 97% rename from raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java rename to raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java index b38c95821..0e1eafedb 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitCallbackRegistryTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java @@ -25,14 +25,14 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -class TraceEmitCallbackRegistryTest { +class TraceEmitTasksPunctuatorTest { private static final long groupingWindowTimeoutMs = 300; private static final TraceIdentity traceIdentity = TraceIdentity.newBuilder() .setTenantId("__default") .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) .build(); - private TraceEmitCallbackRegistry emitCallback; + private TraceEmitTasksPunctuator emitCallback; private KeyValueStore spanStore; private KeyValueStore traceStateStore; @@ -45,7 +45,7 @@ public void setUp() { traceStateStore = mock(KeyValueStore.class); To outputTopicProducer = mock(To.class); emitCallback = - new TraceEmitCallbackRegistry( + new TraceEmitTasksPunctuator( mock(ThrottledPunctuatorConfig.class), mock(KeyValueStore.class), context, From 08cfe4fd50b4551dfd446a7d21075ee9427b26be Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Mon, 9 Oct 2023 14:57:34 +0530 Subject: [PATCH 10/31] fix tests --- .../core/rawspansgrouper/RawSpansGrouper.java | 9 ++-- .../rawspansgrouper/RawSpansProcessor.java | 44 +++++++++++-------- .../rawspansgrouper/RawSpansGrouperTest.java | 43 +++++++++++------- .../raw-spans-grouper/application.conf | 1 + 4 files changed, 57 insertions(+), 40 deletions(-) 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 68f1810f5..67a6abb94 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 @@ -10,7 +10,6 @@ import com.google.common.annotations.VisibleForTesting; import com.typesafe.config.Config; - import java.time.Clock; import java.util.ArrayList; import java.util.List; @@ -47,10 +46,10 @@ public RawSpansGrouper(ConfigClient configClient) { } public StreamsBuilder buildTopology( - Map properties, - StreamsBuilder streamsBuilder, - Map> inputStreams) { - return buildTopologyWithClock(Clock.systemUTC(), properties, streamsBuilder, inputStreams); + Map properties, + StreamsBuilder streamsBuilder, + Map> inputStreams) { + return buildTopologyWithClock(Clock.systemUTC(), properties, streamsBuilder, inputStreams); } @VisibleForTesting 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 6ecc5f2f7..63970054b 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 @@ -50,11 +50,11 @@ import org.slf4j.LoggerFactory; /** - * Receives spans keyed by trace_id and stores them. A {@link TraceEmitTasksPunctuator} is - * scheduled to run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the - * trace. If any spans for the trace arrive within the {@link - * RawSpansProcessor#groupingWindowTimeoutMs} interval then the trace will get an additional {@link - * RawSpansProcessor#groupingWindowTimeoutMs} time to accept spans. + * Receives spans keyed by trace_id and stores them. A {@link TraceEmitTasksPunctuator} is scheduled + * to run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If + * any spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} + * interval then the trace will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} + * time to accept spans. */ public class RawSpansProcessor implements Transformer> { @@ -133,18 +133,25 @@ public void init(ProcessorContext context) { // Punctuator scheduled on stream time => no input messages => no emits will happen // We will almost never have input down to 0, i.e., there are no spans coming to platform, // While using wall clock time handles that case, there is a issue with using wall clock time... - // In cases of lag being burnt, we are processing message produced at different time stamp intervals - // probably faster than at rate which they were produced, now not doing punctuation often will increase the - // amounts of work for punctuator in next iterations and will keep on piling up until lag is burnt completely - // and only then the punctuator will catch up back to normal input rate. This is undesirable, here the outputs - // are only emitted from punctuator, if we burn lag from inputs, we want to push it down to downstream as soon - // as possible, if we hog it more and more it will delay cascading lag to downstream. Given grouper stays at start - // of pipeline it is better to use stream time as using wall clock time can have more undesirable effects + // In cases of lag being burnt, we are processing message produced at different time stamp + // intervals + // probably faster than at rate which they were produced, now not doing punctuation often will + // increase the + // amounts of work for punctuator in next iterations and will keep on piling up until lag is + // burnt completely + // and only then the punctuator will catch up back to normal input rate. This is undesirable, + // here the outputs + // are only emitted from punctuator, if we burn lag from inputs, we want to push it down to + // downstream as soon + // as possible, if we hog it more and more it will delay cascading lag to downstream. Given + // grouper stays at start + // of pipeline it is better to use stream time as using wall clock time can have more + // undesirable effects traceEmitTasksPunctuatorCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), PunctuationType.STREAM_TIME, - traceEmitTasksPunctuator); + traceEmitTasksPunctuator); } public KeyValue transform(TraceIdentity key, RawSpan value) { @@ -179,9 +186,11 @@ public KeyValue transform(TraceIdentity key, Raw traceState.getSpanIds().add(spanId); long prevScheduleTimestamp = traceState.getTraceEndTimestamp(); traceState.setTraceEndTimestamp(currentTimeMs); - if(!traceEmitTasksPunctuator.rescheduleTask( - prevScheduleTimestamp, currentTimeMs + groupingWindowTimeoutMs, key)) { - logger.debug("Failed to reschedule task on getting span for trace key {}, schedule already dropped!", key); + if (!traceEmitTasksPunctuator.rescheduleTask( + prevScheduleTimestamp, currentTimeMs + groupingWindowTimeoutMs, key)) { + logger.debug( + "Failed to reschedule task on getting span for trace key {}, schedule already dropped!", + key); } } @@ -199,8 +208,7 @@ public KeyValue transform(TraceIdentity key, Raw } private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { - int inFlightSpansPerTrace = - traceState != null ? traceState.getSpanIds().size() : Integer.MIN_VALUE; + int inFlightSpansPerTrace = traceState != null ? traceState.getSpanIds().size() : 0; long maxSpanCountTenantLimit = maxSpanCountMap.containsKey(key.getTenantId()) ? maxSpanCountMap.get(key.getTenantId()) 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 d4599b4e2..fcac002a4 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 @@ -151,12 +151,6 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir .setCustomerId("tenant1") .setEvent(createEvent("event-11", "tenant1")) .build(); - RawSpan span20 = - RawSpan.newBuilder() - .setTraceId(ByteBuffer.wrap("trace-3".getBytes())) - .setCustomerId("tenant1") - .setEvent(createEvent("event-20", "tenant1")) - .build(); // create 8 spans for tenant-2 for trace-4 String tenant2 = "tenant2"; @@ -209,19 +203,31 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir .setEvent(createEvent("event-19", tenant2)) .build(); + // dummyTenant is used to advance stream time as required, all spans will be dropped + // because of setting max span count to 0, no traces for this tenant will be emitted + String dummyTenant = "dummyTenant"; + TraceIdentity dummyTraceIdentity = createTraceIdentity(dummyTenant, "dummyTrace"); + RawSpan dummySpan = + RawSpan.newBuilder() + .setTraceId(ByteBuffer.wrap("dummyTrace".getBytes())) + .setCustomerId(tenant2) + .setEvent(createEvent("dummyEvent", dummyTenant)) + .build(); + long messageTime = advanceAndSyncClockMock(0, clock, 0); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span1, messageTime); - advanceAndSyncClockMock(messageTime, clock, 1000); + messageTime = advanceAndSyncClockMock(messageTime, clock, 1000); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span2, messageTime); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span4, messageTime); // select a value < 30s (groupingWindowTimeoutInMs) // this shouldn't trigger a span emit assertTrue(outputTopic.isEmpty()); // the next advance should and emit a trace1 with 2 spans, trace2 with one span - advanceAndSyncClockMock(messageTime, clock, 35_000); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span4, messageTime); + messageTime = advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(dummyTraceIdentity, dummySpan, messageTime); // trace2 should have 1 span span3 StructuredTrace trace = outputTopic.readValue(); @@ -239,11 +245,11 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertTrue(traceEventIds.contains("event-2")); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-1"), span3, messageTime); - advanceAndSyncClockMock(messageTime, clock, 45_000); + messageTime = advanceAndSyncClockMock(messageTime, clock, 45_000); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-2"), span5, messageTime); // the next advance should emit a trace1, trace2 with one span - advanceAndSyncClockMock(messageTime, clock, 35_000); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span6, messageTime); + messageTime = advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(dummyTraceIdentity, dummySpan, messageTime); // trace1 should have 1 span i.e. span3 trace = outputTopic.readValue(); @@ -255,20 +261,22 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir assertEquals(1, trace.getEventList().size()); assertEquals("event-5", new String(trace.getEventList().get(0).getEventId().array())); + inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span6, messageTime); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span7, messageTime); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span8, messageTime); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span9, messageTime); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span10, messageTime); inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span11, messageTime); // the next advance should emit trace3 - advanceAndSyncClockMock(messageTime, clock, 35_000); - inputTopic.pipeInput(createTraceIdentity(tenantId, "trace-3"), span20, messageTime); + messageTime = advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(dummyTraceIdentity, dummySpan, messageTime); // trace3 should be truncated with 5 spans because of tenant limit trace = outputTopic.readValue(); assertEquals(5, trace.getEventList().size()); - // input 8 spans of trace-4 for tenant2, as there is global upper limit apply, it will emit only 6 + // input 8 spans of trace-4 for tenant2, as there is global upper limit apply, it will emit only + // 6 inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span12, messageTime); inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span13, messageTime); inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span14, messageTime); @@ -276,9 +284,10 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span16, messageTime); inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span17, messageTime); inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span18, messageTime); - // the next advance should emit trace 4 - advanceAndSyncClockMock(messageTime, clock, 35_000); inputTopic.pipeInput(createTraceIdentity(tenant2, "trace-4"), span19, messageTime); + // the next advance should emit trace 4 + messageTime = advanceAndSyncClockMock(messageTime, clock, 35_000); + inputTopic.pipeInput(dummyTraceIdentity, dummySpan, messageTime); TestRecord testRecord = outputTopic.readRecord(); 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 e250a3616..693432447 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 @@ -37,6 +37,7 @@ span.window.store.segment.size.mins = ${?SPAN_WINDOW_STORE_SEGMENT_SIZE_MINS} default.max.span.count = 6 max.span.count = { tenant1 = 5 + dummyTenant = 0 } span.groupby.session.window.interval = 30 From 0883670e2602a12f52b54cff534c7840dd571073 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Mon, 9 Oct 2023 15:06:33 +0530 Subject: [PATCH 11/31] nit --- .../core/rawspansgrouper/RawSpansGrouper.java | 4 +-- ...rocessor.java => RawSpansTransformer.java} | 35 ++++++++----------- .../TraceEmitTasksPunctuator.java | 2 +- .../rawspansgrouper/RawSpansGrouperTest.java | 2 ++ 4 files changed, 20 insertions(+), 23 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{RawSpansProcessor.java => RawSpansTransformer.java} (90%) 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 67a6abb94..184066e77 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 @@ -112,8 +112,8 @@ StreamsBuilder buildTopologyWithClock( inputStream .transform( - () -> new RawSpansProcessor(clock), - Named.as(RawSpansProcessor.class.getSimpleName()), + () -> new RawSpansTransformer(clock), + Named.as(RawSpansTransformer.class.getSimpleName()), SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, TRACE_EMIT_CALLBACK_REGISTRY_STORE_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/RawSpansTransformer.java similarity index 90% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java index 63970054b..f2f1e55bd 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/RawSpansTransformer.java @@ -51,15 +51,15 @@ /** * Receives spans keyed by trace_id and stores them. A {@link TraceEmitTasksPunctuator} is scheduled - * to run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If - * any spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} - * interval then the trace will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} - * time to accept spans. + * to run after the {@link RawSpansTransformer#groupingWindowTimeoutMs} interval to emit the trace. + * If any spans for the trace arrive within the {@link RawSpansTransformer#groupingWindowTimeoutMs} + * interval then the trace will get an additional {@link + * RawSpansTransformer#groupingWindowTimeoutMs} time to accept spans. */ -public class RawSpansProcessor +public class RawSpansTransformer implements Transformer> { - private static final Logger logger = LoggerFactory.getLogger(RawSpansProcessor.class); + private static final Logger logger = LoggerFactory.getLogger(RawSpansTransformer.class); private static final String PROCESSING_LATENCY_TIMER = "hypertrace.rawspansgrouper.processing.latency"; private static final ConcurrentMap tenantToSpansGroupingTimer = @@ -82,7 +82,7 @@ public class RawSpansProcessor private TraceEmitTasksPunctuator traceEmitTasksPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; - public RawSpansProcessor(Clock clock) { + public RawSpansTransformer(Clock clock) { this.clock = clock; } @@ -132,21 +132,16 @@ public void init(ProcessorContext context) { dataflowSamplingPercent); // Punctuator scheduled on stream time => no input messages => no emits will happen // We will almost never have input down to 0, i.e., there are no spans coming to platform, - // While using wall clock time handles that case, there is a issue with using wall clock time... + // While using wall clock time handles that case, there is an issue with using wall clock. // In cases of lag being burnt, we are processing message produced at different time stamp - // intervals - // probably faster than at rate which they were produced, now not doing punctuation often will - // increase the - // amounts of work for punctuator in next iterations and will keep on piling up until lag is - // burnt completely - // and only then the punctuator will catch up back to normal input rate. This is undesirable, - // here the outputs - // are only emitted from punctuator, if we burn lag from inputs, we want to push it down to - // downstream as soon + // intervals, probably at higher rate than which they were produced, now not doing punctuation + // often will increase the amount of work yielding punctuator in next iterations and will keep + // on piling up until lag is burnt completely and only then the punctuator will catch up back to + // normal input rate. This is undesirable, here the outputs are only emitted from punctuator. + // If we burn lag from input topic, we want to push it down to output & downstream as soon // as possible, if we hog it more and more it will delay cascading lag to downstream. Given - // grouper stays at start - // of pipeline it is better to use stream time as using wall clock time can have more - // undesirable effects + // grouper stays at start of pipeline and also that input dying down almost never happens + // it is better to use stream time over wall clock time for yielding trace emit tasks punctuator traceEmitTasksPunctuatorCancellable = context.schedule( jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java index e7226b3be..80650cbe7 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java @@ -43,7 +43,7 @@ /** * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link - * RawSpansProcessor#groupingWindowTimeoutMs} + * RawSpansTransformer#groupingWindowTimeoutMs} */ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator { 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 fcac002a4..04c1d6d1c 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 @@ -223,6 +223,8 @@ public void whenRawSpansAreReceivedWithInactivityExpectTraceToBeOutput(@TempDir // select a value < 30s (groupingWindowTimeoutInMs) // this shouldn't trigger a span emit + messageTime = advanceAndSyncClockMock(messageTime, clock, 200); + inputTopic.pipeInput(dummyTraceIdentity, dummySpan, messageTime); assertTrue(outputTopic.isEmpty()); // the next advance should and emit a trace1 with 2 spans, trace2 with one span From cbab5466f807a1120181550bf492ae2b6aeb1e26 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Mon, 9 Oct 2023 16:00:18 +0530 Subject: [PATCH 12/31] nit --- .../TraceEmitTasksPunctuator.java | 16 ++++++++-------- .../TraceEmitTasksPunctuatorTest.java | 6 +++--- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java index 80650cbe7..c442bbd8b 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java @@ -31,9 +31,9 @@ import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.AbstractThrottledPunctuator; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.DropScheduleAction; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.RescheduleAction; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.ScheduleAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.DropTask; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.ReschduleTask; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.TaskResult; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -67,7 +67,7 @@ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); - private static final DropScheduleAction DROP_SCHEDULE_ACTION = new DropScheduleAction(); + private static final DropTask DROP_TASK = new DropTask(); private final double dataflowSamplingPercent; private final ProcessorContext context; private final KeyValueStore spanStore; @@ -93,7 +93,7 @@ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator= groupingWindowTimeoutMs) { // Implies that no new spans for the trace have arrived within the last @@ -110,9 +110,9 @@ protected ScheduleAction callback(long punctuateTimestamp, TraceIdentity key) { // so the trace can be finalized and emitted emitTrace(key, traceState); // no need of running again for this - return DROP_SCHEDULE_ACTION; + return DROP_TASK; } - return new RescheduleAction(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); + return new ReschduleTask(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); } private void emitTrace(TraceIdentity key, TraceState traceState) { diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java index 0e1eafedb..3f699b344 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java @@ -17,7 +17,7 @@ import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.ScheduleAction; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.TaskResult; import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; import org.hypertrace.core.spannormalizer.SpanIdentity; import org.hypertrace.core.spannormalizer.TraceIdentity; @@ -69,7 +69,7 @@ public void testWhenWindowIsExtended() { .build(); when(traceStateStore.get(eq(traceIdentity))).thenReturn(traceState); - ScheduleAction callbackAction = emitCallback.callback(300, traceIdentity); + TaskResult callbackAction = emitCallback.executeTask(300, traceIdentity); assertEquals( traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs, callbackAction.getRescheduleTimestamp().get()); @@ -102,7 +102,7 @@ public void testWhenTraceToBeEmitted() { .build(); when(spanStore.delete(any())).thenReturn(rawSpan); - assertTrue(emitCallback.callback(450, traceIdentity).getRescheduleTimestamp().isEmpty()); + assertTrue(emitCallback.executeTask(450, traceIdentity).getRescheduleTimestamp().isEmpty()); verify(traceStateStore, times(1)).get(traceIdentity); verify(spanStore, times(1)).delete(any()); From 6cca310d0da84e694ceebe6fcb016aad3d644385 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Wed, 11 Oct 2023 18:35:42 +0530 Subject: [PATCH 13/31] nit --- .../rawspansgrouper/TraceEmitTasksPunctuator.java | 12 ++++++------ .../main/resources/configs/common/application.conf | 3 ++- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java index c442bbd8b..a6ed49ce1 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java @@ -31,8 +31,8 @@ import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.AbstractThrottledPunctuator; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.DropTask; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.ReschduleTask; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.CompletedTask; +import org.hypertrace.core.kafkastreams.framework.punctuators.action.RescheduleTask; import org.hypertrace.core.kafkastreams.framework.punctuators.action.TaskResult; import org.hypertrace.core.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; @@ -67,7 +67,7 @@ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); - private static final DropTask DROP_TASK = new DropTask(); + private static final CompletedTask COMPLETED_TASK = new CompletedTask(); private final double dataflowSamplingPercent; private final ProcessorContext context; private final KeyValueStore spanStore; @@ -102,7 +102,7 @@ protected TaskResult executeTask(long punctuateTimestamp, TraceIdentity key) { // "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", // key.getTenantId(), // HexUtils.getHex(key.getTraceId())); - return DROP_TASK; + return COMPLETED_TASK; } if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { // Implies that no new spans for the trace have arrived within the last @@ -110,9 +110,9 @@ protected TaskResult executeTask(long punctuateTimestamp, TraceIdentity key) { // so the trace can be finalized and emitted emitTrace(key, traceState); // no need of running again for this - return DROP_TASK; + return COMPLETED_TASK; } - return new ReschduleTask(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); + return new RescheduleTask(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); } private void emitTrace(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 3d6baa6e0..5736e907f 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 @@ -62,8 +62,9 @@ metrics.reporter { dataflow.metriccollection.sampling.percent = 10.0 trace.emit.callback.registry { - frequency = 30s + frequency = 5s frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} + // TODO: yield and window won't be there here, pushed down to defaults in kafka streams yield = 15s yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} window = 1ms From 9519edd80a8ea2450b354f344cc041b682073cb1 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Thu, 12 Oct 2023 18:01:14 +0530 Subject: [PATCH 14/31] update --- .../RawSpanGrouperConstants.java | 14 ++++---- .../core/rawspansgrouper/RawSpansGrouper.java | 6 ++-- .../rawspansgrouper/RawSpansTransformer.java | 33 +++++++++---------- ...nctuator.java => TraceEmitPunctuator.java} | 26 +++++++-------- .../resources/configs/common/application.conf | 9 ++--- ...Test.java => TraceEmitPunctuatorTest.java} | 6 ++-- .../raw-spans-grouper/application.conf | 7 ++-- 7 files changed, 45 insertions(+), 56 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{TraceEmitTasksPunctuator.java => TraceEmitPunctuator.java} (93%) rename raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/{TraceEmitTasksPunctuatorTest.java => TraceEmitPunctuatorTest.java} (97%) 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 ab040257e..4eea2619e 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 @@ -17,12 +17,10 @@ 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"; - static final String TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME = - "test-kishan-trace-emit-callback-registry-store"; // TODO: remove test after done with it - static final String TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY = - "trace.emit.callback.registry.frequency"; - static final String TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY = - "trace.emit.callback.registry.yield"; - static final String TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY = - "trace.emit.callback.registry.window"; + static final String TRACE_EMIT_PUNCTUATOR = "trace-emit-punctuator"; + static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = + "test-kishan-trace-emit-callback-registry-store"; // TODO: remove test after done with it, and + // rename without callback registry + static final String TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY = + "trace.emit.punctuator.frequency"; } 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 184066e77..ee35272c1 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 @@ -5,7 +5,7 @@ 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_STATE_STORE_NAME; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR_STORE_NAME; import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_STATE_STORE; import com.google.common.annotations.VisibleForTesting; @@ -88,7 +88,7 @@ StreamsBuilder buildTopologyWithClock( StoreBuilder> traceEmitCallbackRegistryStoreBuilder = Stores.keyValueStoreBuilder( - Stores.persistentKeyValueStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME), + Stores.persistentKeyValueStore(TRACE_EMIT_PUNCTUATOR_STORE_NAME), Serdes.Long(), Serdes.ListSerde(ArrayList.class, valueSerde)) .withCachingEnabled(); @@ -116,7 +116,7 @@ StreamsBuilder buildTopologyWithClock( Named.as(RawSpansTransformer.class.getSimpleName()), SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, - TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME) + TRACE_EMIT_PUNCTUATOR_STORE_NAME) .to(outputTopic, outputTopicProducer); return streamsBuilder; diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java index f2f1e55bd..6eee49442 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java @@ -1,6 +1,7 @@ package org.hypertrace.core.rawspansgrouper; 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.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; @@ -9,10 +10,9 @@ 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; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY; -import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR; +import static org.hypertrace.core.rawspansgrouper.RawSpanGrouperConstants.TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY; +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; @@ -50,9 +50,9 @@ import org.slf4j.LoggerFactory; /** - * Receives spans keyed by trace_id and stores them. A {@link TraceEmitTasksPunctuator} is scheduled - * to run after the {@link RawSpansTransformer#groupingWindowTimeoutMs} interval to emit the trace. - * If any spans for the trace arrive within the {@link RawSpansTransformer#groupingWindowTimeoutMs} + * Receives spans keyed by trace_id and stores them. A {@link TraceEmitPunctuator} is scheduled to + * run after the {@link RawSpansTransformer#groupingWindowTimeoutMs} interval to emit the trace. If + * any spans for the trace arrive within the {@link RawSpansTransformer#groupingWindowTimeoutMs} * interval then the trace will get an additional {@link * RawSpansTransformer#groupingWindowTimeoutMs} time to accept spans. */ @@ -79,7 +79,7 @@ public class RawSpansTransformer private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; - private TraceEmitTasksPunctuator traceEmitTasksPunctuator; + private TraceEmitPunctuator traceEmitPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; public RawSpansTransformer(Clock clock) { @@ -117,12 +117,11 @@ public void init(ProcessorContext context) { this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); KeyValueStore> traceEmitCallbackRegistryStore = - context.getStateStore(TRACE_EMIT_CALLBACK_REGISTRY_STORE_NAME); - traceEmitTasksPunctuator = - new TraceEmitTasksPunctuator( + context.getStateStore(TRACE_EMIT_PUNCTUATOR_STORE_NAME); + traceEmitPunctuator = + new TraceEmitPunctuator( new ThrottledPunctuatorConfig( - jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_YIELD_CONFIG_KEY).toMillis(), - jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_WINDOW_CONFIG_KEY).toMillis()), + jobConfig.getConfig(KAFKA_STREAMS_CONFIG_KEY), TRACE_EMIT_PUNCTUATOR), traceEmitCallbackRegistryStore, context, spanStore, @@ -144,9 +143,9 @@ public void init(ProcessorContext context) { // it is better to use stream time over wall clock time for yielding trace emit tasks punctuator traceEmitTasksPunctuatorCancellable = context.schedule( - jobConfig.getDuration(TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY_CONFIG_KEY), + jobConfig.getDuration(TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY), PunctuationType.STREAM_TIME, - traceEmitTasksPunctuator); + traceEmitPunctuator); } public KeyValue transform(TraceIdentity key, RawSpan value) { @@ -176,12 +175,12 @@ public KeyValue transform(TraceIdentity key, Raw .setTraceId(traceId) .setSpanIds(List.of(spanId)) .build(); - traceEmitTasksPunctuator.scheduleTask(currentTimeMs, key); + traceEmitPunctuator.scheduleTask(currentTimeMs, key); } else { traceState.getSpanIds().add(spanId); long prevScheduleTimestamp = traceState.getTraceEndTimestamp(); traceState.setTraceEndTimestamp(currentTimeMs); - if (!traceEmitTasksPunctuator.rescheduleTask( + if (!traceEmitPunctuator.rescheduleTask( prevScheduleTimestamp, currentTimeMs + groupingWindowTimeoutMs, key)) { logger.debug( "Failed to reschedule task on getting span for trace key {}, schedule already dropped!", diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java similarity index 93% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java index a6ed49ce1..1c3cf6f03 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuator.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuator.java @@ -31,8 +31,8 @@ import org.hypertrace.core.datamodel.shared.trace.StructuredTraceBuilder; import org.hypertrace.core.kafkastreams.framework.punctuators.AbstractThrottledPunctuator; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.CompletedTask; -import org.hypertrace.core.kafkastreams.framework.punctuators.action.RescheduleTask; +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.serviceframework.metrics.PlatformMetricsRegistry; import org.hypertrace.core.spannormalizer.SpanIdentity; @@ -45,9 +45,9 @@ * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansTransformer#groupingWindowTimeoutMs} */ -class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator { +class TraceEmitPunctuator extends AbstractThrottledPunctuator { - private static final Logger logger = LoggerFactory.getLogger(TraceEmitTasksPunctuator.class); + private static final Logger logger = LoggerFactory.getLogger(TraceEmitPunctuator.class); private static final Object mutex = new Object(); private static final Timer spansGrouperArrivalLagTimer = @@ -67,7 +67,7 @@ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator tenantToTraceWithDuplicateSpansCounter = new ConcurrentHashMap<>(); - private static final CompletedTask COMPLETED_TASK = new CompletedTask(); + private static final CompletedTaskResult COMPLETED_TASK_RESULT = new CompletedTaskResult(); private final double dataflowSamplingPercent; private final ProcessorContext context; private final KeyValueStore spanStore; @@ -75,7 +75,7 @@ class TraceEmitTasksPunctuator extends AbstractThrottledPunctuator> callbackRegistryStore, ProcessorContext context, @@ -98,11 +98,11 @@ protected TaskResult executeTask(long punctuateTimestamp, TraceIdentity key) { if (null == traceState || null == traceState.getSpanIds() || traceState.getSpanIds().isEmpty()) { - // logger.warn( - // "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", - // key.getTenantId(), - // HexUtils.getHex(key.getTraceId())); - return COMPLETED_TASK; + logger.warn( + "TraceState for tenant_id=[{}], trace_id=[{}] is missing.", + key.getTenantId(), + HexUtils.getHex(key.getTraceId())); + return COMPLETED_TASK_RESULT; } if (punctuateTimestamp - traceState.getTraceEndTimestamp() >= groupingWindowTimeoutMs) { // Implies that no new spans for the trace have arrived within the last @@ -110,9 +110,9 @@ protected TaskResult executeTask(long punctuateTimestamp, TraceIdentity key) { // so the trace can be finalized and emitted emitTrace(key, traceState); // no need of running again for this - return COMPLETED_TASK; + return COMPLETED_TASK_RESULT; } - return new RescheduleTask(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); + return new RescheduleTaskResult(traceState.getTraceEndTimestamp() + groupingWindowTimeoutMs); } private void emitTrace(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 5736e907f..64f47ac24 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 @@ -61,12 +61,7 @@ metrics.reporter { dataflow.metriccollection.sampling.percent = 10.0 -trace.emit.callback.registry { +trace.emit.punctuator { frequency = 5s - frequency = ${?TRACE_EMIT_CALLBACK_REGISTRY_FREQUENCY} - // TODO: yield and window won't be there here, pushed down to defaults in kafka streams - yield = 15s - yield = ${?TRACE_EMIT_CALLBACK_REGISTRY_YIELD} - window = 1ms - window = ${?TRACE_EMIT_CALLBACK_REGISTRY_WINDOW} + frequency = ${?TRACE_EMIT_PUNCTUATOR_FREQUENCY} } \ No newline at end of file diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java similarity index 97% rename from raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java rename to raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java index 3f699b344..9e817c506 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitTasksPunctuatorTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java @@ -25,14 +25,14 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -class TraceEmitTasksPunctuatorTest { +class TraceEmitPunctuatorTest { private static final long groupingWindowTimeoutMs = 300; private static final TraceIdentity traceIdentity = TraceIdentity.newBuilder() .setTenantId("__default") .setTraceId(ByteBuffer.wrap("trace-1".getBytes())) .build(); - private TraceEmitTasksPunctuator emitCallback; + private TraceEmitPunctuator emitCallback; private KeyValueStore spanStore; private KeyValueStore traceStateStore; @@ -45,7 +45,7 @@ public void setUp() { traceStateStore = mock(KeyValueStore.class); To outputTopicProducer = mock(To.class); emitCallback = - new TraceEmitTasksPunctuator( + new TraceEmitPunctuator( mock(ThrottledPunctuatorConfig.class), mock(KeyValueStore.class), context, 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 693432447..6ecac3b23 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 @@ -11,6 +11,7 @@ precreate.topics = false kafka.streams.config = { application.id = raw-spans-to-structured-traces-grouping-job num.stream.threads = 2 + consumer.session.timeout.ms = 300000 topic.cleanup.policy = "delete,compact" replication.factor = 3 bootstrap.servers = "localhost:9092" @@ -46,8 +47,4 @@ span.groupby.session.window.graceperiod.ms = 100 dataflow.metriccollection.sampling.percent = 10.0 -trace.emit.callback.registry { - frequency = 15s - yield = 5s - window = 5ms -} \ No newline at end of file +trace.emit.punctuator.frequency = 15s \ No newline at end of file From 93c48ff849e971fda3dd81f5345c9f1793944b99 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 16:32:47 +0530 Subject: [PATCH 15/31] nit --- gradle/libs.versions.toml | 2 +- .../helm/templates/raw-spans-grouper-config.yaml | 8 ++------ raw-spans-grouper/helm/values.yaml | 5 +---- .../core/rawspansgrouper/RawSpanGrouperConstants.java | 4 +--- .../hypertrace/core/rawspansgrouper/RawSpansGrouper.java | 4 ++-- .../core/rawspansgrouper/RawSpansTransformer.java | 4 ++-- .../core/rawspansgrouper/TraceEmitPunctuator.java | 6 +++--- 7 files changed, 12 insertions(+), 21 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 9e570a508..41c3bccee 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -3,7 +3,7 @@ hypertrace-entity-service = "0.8.78" hypertrace-config-service = "0.1.54" hypertrace-grpc-utils = "0.12.4" hypertrace-serviceFramework = "0.1.60" -hypertrace-kafkaStreams = "0.4.0-SNAPSHOT" +hypertrace-kafkaStreams = "0.4.0" hypertrace-view-generator = "0.4.19" grpc = "1.57.2" 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 eb86ba4bc..ca65a46e4 100644 --- a/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml +++ b/raw-spans-grouper/helm/templates/raw-spans-grouper-config.yaml @@ -82,10 +82,6 @@ data: } {{- end }} - {{- if hasKey .Values.rawSpansGrouperConfig "traceEmitCallbackRegistry" }} - trace.emit.callback.registry { - frequency = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.frequency }} - yield = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.yield }} - window = {{ .Values.rawSpansGrouperConfig.traceEmitCallbackRegistry.window }} - } + {{- if hasKey .Values.rawSpansGrouperConfig "traceEmitPunctuatorFrequency" }} + trace.emit.punctuator.frequency = {{ .Values.rawSpansGrouperConfig.traceEmitPunctuatorFrequency }} {{- end }} diff --git a/raw-spans-grouper/helm/values.yaml b/raw-spans-grouper/helm/values.yaml index 3f01ae776..139c0855c 100644 --- a/raw-spans-grouper/helm/values.yaml +++ b/raw-spans-grouper/helm/values.yaml @@ -132,10 +132,7 @@ rawSpansGrouperConfig: groupPartitionerEnabled: false groupPartitionerConfigServiceHost: config-service groupPartitionerConfigServicePort: 50104 - traceEmitCallbackRegistry: - frequency: 1s - yield: 10s - window: 5ms + traceEmitPunctuatorFrequency: 5s 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 4eea2619e..71d6d9ffa 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,9 +18,7 @@ public class RawSpanGrouperConstants { public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; public static final String TRUNCATED_TRACES_COUNTER = "hypertrace.truncated.traces"; static final String TRACE_EMIT_PUNCTUATOR = "trace-emit-punctuator"; - static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = - "test-kishan-trace-emit-callback-registry-store"; // TODO: remove test after done with it, and - // rename without callback registry + static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = "trace-emit-punctuator-store"; static final String TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY = "trace.emit.punctuator.frequency"; } 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 ee35272c1..b195cb2eb 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 @@ -86,7 +86,7 @@ StreamsBuilder buildTopologyWithClock( Stores.persistentKeyValueStore(SPAN_STATE_STORE_NAME), keySerde, valueSerde) .withCachingEnabled(); - StoreBuilder> traceEmitCallbackRegistryStoreBuilder = + StoreBuilder> traceEmitPunctuatorStoreBuilder = Stores.keyValueStoreBuilder( Stores.persistentKeyValueStore(TRACE_EMIT_PUNCTUATOR_STORE_NAME), Serdes.Long(), @@ -95,7 +95,7 @@ StreamsBuilder buildTopologyWithClock( streamsBuilder.addStateStore(spanStoreBuilder); streamsBuilder.addStateStore(traceStateStoreBuilder); - streamsBuilder.addStateStore(traceEmitCallbackRegistryStoreBuilder); + streamsBuilder.addStateStore(traceEmitPunctuatorStoreBuilder); StreamPartitioner groupPartitioner = new GroupPartitionerBuilder() diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java index 6eee49442..915d4a4ab 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java @@ -116,13 +116,13 @@ public void init(ProcessorContext context) { this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); - KeyValueStore> traceEmitCallbackRegistryStore = + KeyValueStore> traceEmitPunctuatorStore = context.getStateStore(TRACE_EMIT_PUNCTUATOR_STORE_NAME); traceEmitPunctuator = new TraceEmitPunctuator( new ThrottledPunctuatorConfig( jobConfig.getConfig(KAFKA_STREAMS_CONFIG_KEY), TRACE_EMIT_PUNCTUATOR), - traceEmitCallbackRegistryStore, + traceEmitPunctuatorStore, context, spanStore, traceStateStore, 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 1c3cf6f03..e31b27d35 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 @@ -42,7 +42,7 @@ import org.slf4j.LoggerFactory; /** - * Callbacks to check if a trace can be finalized and emitted based on inactivity period of {@link + * Check if a trace can be finalized and emitted based on inactivity period of {@link * RawSpansTransformer#groupingWindowTimeoutMs} */ class TraceEmitPunctuator extends AbstractThrottledPunctuator { @@ -77,14 +77,14 @@ class TraceEmitPunctuator extends AbstractThrottledPunctuator { TraceEmitPunctuator( ThrottledPunctuatorConfig throttledPunctuatorConfig, - KeyValueStore> callbackRegistryStore, + KeyValueStore> throttledPunctuatorStore, ProcessorContext context, KeyValueStore spanStore, KeyValueStore traceStateStore, To outputTopicProducer, long groupingWindowTimeoutMs, double dataflowSamplingPercent) { - super(Clock.systemUTC(), throttledPunctuatorConfig, callbackRegistryStore); + super(Clock.systemUTC(), throttledPunctuatorConfig, throttledPunctuatorStore); this.context = context; this.spanStore = spanStore; this.traceStateStore = traceStateStore; From 5d87b72e31549efb3332deb15639be1d78a8e613 Mon Sep 17 00:00:00 2001 From: Laxman Ch Date: Fri, 13 Oct 2023 19:26:27 +0530 Subject: [PATCH 16/31] minor refactor to migrate to kafka streams new apis --- .../RawSpanGrouperConstants.java | 2 +- .../core/rawspansgrouper/RawSpansGrouper.java | 6 +-- ...ransformer.java => RawSpansProcessor.java} | 43 +++++++++---------- .../rawspansgrouper/TraceEmitPunctuator.java | 18 ++++---- .../TraceEmitPunctuatorTest.java | 7 +-- 5 files changed, 39 insertions(+), 37 deletions(-) rename raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/{RawSpansTransformer.java => RawSpansProcessor.java} (90%) 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 71d6d9ffa..7c91a30d7 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,7 +18,7 @@ public class RawSpanGrouperConstants { public static final String DROPPED_SPANS_COUNTER = "hypertrace.dropped.spans"; public static final String TRUNCATED_TRACES_COUNTER = "hypertrace.truncated.traces"; static final String TRACE_EMIT_PUNCTUATOR = "trace-emit-punctuator"; - static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = "trace-emit-punctuator-store"; + static final String TRACE_EMIT_PUNCTUATOR_STORE_NAME = "emitter-store"; static final String TRACE_EMIT_PUNCTUATOR_FREQUENCY_CONFIG_KEY = "trace.emit.punctuator.frequency"; } 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 b195cb2eb..aaa89e693 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 @@ -111,9 +111,9 @@ StreamsBuilder buildTopologyWithClock( outputTopicProducer = outputTopicProducer.withName(OUTPUT_TOPIC_PRODUCER); inputStream - .transform( - () -> new RawSpansTransformer(clock), - Named.as(RawSpansTransformer.class.getSimpleName()), + .process( + () -> new RawSpansProcessor(clock), + Named.as(RawSpansProcessor.class.getSimpleName()), SPAN_STATE_STORE_NAME, TRACE_STATE_STORE, TRACE_EMIT_PUNCTUATOR_STORE_NAME) diff --git a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java similarity index 90% rename from raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java rename to raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java index 915d4a4ab..885b023f4 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansTransformer.java +++ b/raw-spans-grouper/raw-spans-grouper/src/main/java/org/hypertrace/core/rawspansgrouper/RawSpansProcessor.java @@ -31,12 +31,11 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.processor.Cancellable; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; -import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Processor; +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.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -51,15 +50,15 @@ /** * Receives spans keyed by trace_id and stores them. A {@link TraceEmitPunctuator} is scheduled to - * run after the {@link RawSpansTransformer#groupingWindowTimeoutMs} interval to emit the trace. If - * any spans for the trace arrive within the {@link RawSpansTransformer#groupingWindowTimeoutMs} - * interval then the trace will get an additional {@link - * RawSpansTransformer#groupingWindowTimeoutMs} time to accept spans. + * run after the {@link RawSpansProcessor#groupingWindowTimeoutMs} interval to emit the trace. If + * any spans for the trace arrive within the {@link RawSpansProcessor#groupingWindowTimeoutMs} + * interval then the trace will get an additional {@link RawSpansProcessor#groupingWindowTimeoutMs} + * time to accept spans. */ -public class RawSpansTransformer - implements Transformer> { +public class RawSpansProcessor + implements Processor { - private static final Logger logger = LoggerFactory.getLogger(RawSpansTransformer.class); + private static final Logger logger = LoggerFactory.getLogger(RawSpansProcessor.class); private static final String PROCESSING_LATENCY_TIMER = "hypertrace.rawspansgrouper.processing.latency"; private static final ConcurrentMap tenantToSpansGroupingTimer = @@ -75,19 +74,18 @@ public class RawSpansTransformer private KeyValueStore spanStore; private KeyValueStore traceStateStore; private long groupingWindowTimeoutMs; - private To outputTopic; private double dataflowSamplingPercent = -1; private static final Map maxSpanCountMap = new HashMap<>(); private long defaultMaxSpanCountLimit = Long.MAX_VALUE; private TraceEmitPunctuator traceEmitPunctuator; private Cancellable traceEmitTasksPunctuatorCancellable; - public RawSpansTransformer(Clock clock) { + public RawSpansProcessor(Clock clock) { this.clock = clock; } @Override - public void init(ProcessorContext context) { + public void init(ProcessorContext context) { this.spanStore = context.getStateStore(SPAN_STATE_STORE_NAME); this.traceStateStore = context.getStateStore(TRACE_STATE_STORE); Config jobConfig = (Config) (context.appConfigs().get(RAW_SPANS_GROUPER_JOB_CONFIG)); @@ -114,8 +112,6 @@ public void init(ProcessorContext context) { defaultMaxSpanCountLimit = jobConfig.getLong(DEFAULT_INFLIGHT_TRACE_MAX_SPAN_COUNT); } - this.outputTopic = To.child(OUTPUT_TOPIC_PRODUCER); - KeyValueStore> traceEmitPunctuatorStore = context.getStateStore(TRACE_EMIT_PUNCTUATOR_STORE_NAME); traceEmitPunctuator = @@ -126,7 +122,7 @@ public void init(ProcessorContext context) { context, spanStore, traceStateStore, - outputTopic, + OUTPUT_TOPIC_PRODUCER, groupingWindowTimeoutMs, dataflowSamplingPercent); // Punctuator scheduled on stream time => no input messages => no emits will happen @@ -148,16 +144,18 @@ public void init(ProcessorContext context) { traceEmitPunctuator); } - public KeyValue transform(TraceIdentity key, RawSpan value) { + @Override + public void process(Record record) { Instant start = Instant.now(); long currentTimeMs = clock.millis(); + TraceIdentity key = record.key(); + RawSpan value = record.value(); TraceState traceState = traceStateStore.get(key); boolean firstEntry = (traceState == null); - ByteBuffer debugSpanId = value.getEvent().getEventId(); if (shouldDropSpan(key, traceState)) { - return null; + return; } String tenantId = key.getTenantId(); @@ -197,8 +195,9 @@ public KeyValue transform(TraceIdentity key, Raw PlatformMetricsRegistry.registerTimer( PROCESSING_LATENCY_TIMER, Map.of("tenantId", k))) .record(Duration.between(start, Instant.now()).toMillis(), TimeUnit.MILLISECONDS); - // the punctuator will emit the trace - return null; + // no need to do context.forward. the punctuator will emit the trace once it's eligible to be + // emitted + return; } private boolean shouldDropSpan(TraceIdentity key, TraceState traceState) { 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 e31b27d35..28b6f81cb 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 @@ -19,8 +19,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.To; +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.RawSpan; import org.hypertrace.core.datamodel.StructuredTrace; @@ -43,7 +43,7 @@ /** * Check if a trace can be finalized and emitted based on inactivity period of {@link - * RawSpansTransformer#groupingWindowTimeoutMs} + * RawSpansProcessor#groupingWindowTimeoutMs} */ class TraceEmitPunctuator extends AbstractThrottledPunctuator { @@ -69,19 +69,19 @@ class TraceEmitPunctuator extends AbstractThrottledPunctuator { private static final CompletedTaskResult COMPLETED_TASK_RESULT = new CompletedTaskResult(); private final double dataflowSamplingPercent; - private final ProcessorContext context; + private final ProcessorContext context; private final KeyValueStore spanStore; private final KeyValueStore traceStateStore; - private final To outputTopicProducer; + private final String outputTopicProducer; private final long groupingWindowTimeoutMs; TraceEmitPunctuator( ThrottledPunctuatorConfig throttledPunctuatorConfig, KeyValueStore> throttledPunctuatorStore, - ProcessorContext context, + ProcessorContext context, KeyValueStore spanStore, KeyValueStore traceStateStore, - To outputTopicProducer, + String outputTopicProducer, long groupingWindowTimeoutMs, double dataflowSamplingPercent) { super(Clock.systemUTC(), throttledPunctuatorConfig, throttledPunctuatorStore); @@ -192,7 +192,9 @@ private void emitTrace(TraceIdentity key, TraceState traceState) { TRACES_EMITTER_COUNTER, Map.of("tenantId", k))) .increment(); - context.forward(key, trace, outputTopicProducer); + context.forward( + new Record(key, trace, System.currentTimeMillis()), + outputTopicProducer); } private Timestamps trackEndToEndLatencyTimestamps( diff --git a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java index 9e817c506..eb4f39030 100644 --- a/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java +++ b/raw-spans-grouper/raw-spans-grouper/src/test/java/org/hypertrace/core/rawspansgrouper/TraceEmitPunctuatorTest.java @@ -11,11 +11,12 @@ import java.nio.ByteBuffer; import java.util.List; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.state.KeyValueStore; import org.hypertrace.core.datamodel.Event; import org.hypertrace.core.datamodel.RawSpan; +import org.hypertrace.core.datamodel.StructuredTrace; import org.hypertrace.core.kafkastreams.framework.punctuators.ThrottledPunctuatorConfig; import org.hypertrace.core.kafkastreams.framework.punctuators.action.TaskResult; import org.hypertrace.core.kafkastreams.framework.serdes.AvroSerde; @@ -39,7 +40,7 @@ class TraceEmitPunctuatorTest { @BeforeEach public void setUp() { AvroSerde avroSerde = new AvroSerde(); - ProcessorContext context = mock(ProcessorContext.class); + ProcessorContext context = mock(ProcessorContext.class); when(context.keySerde()).thenReturn(avroSerde); spanStore = mock(KeyValueStore.class); traceStateStore = mock(KeyValueStore.class); @@ -51,7 +52,7 @@ public void setUp() { context, spanStore, traceStateStore, - outputTopicProducer, + RawSpanGrouperConstants.OUTPUT_TOPIC_PRODUCER, groupingWindowTimeoutMs, -1); } From 90281c3701e2e5915faf48b497d2200573d96903 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 19:56:49 +0530 Subject: [PATCH 17/31] Nit --- .../org/hypertrace/ingester/HypertraceIngesterTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/hypertrace-ingester/src/test/java/org/hypertrace/ingester/HypertraceIngesterTest.java b/hypertrace-ingester/src/test/java/org/hypertrace/ingester/HypertraceIngesterTest.java index c2b17280c..e38d417e9 100644 --- a/hypertrace-ingester/src/test/java/org/hypertrace/ingester/HypertraceIngesterTest.java +++ b/hypertrace-ingester/src/test/java/org/hypertrace/ingester/HypertraceIngesterTest.java @@ -9,7 +9,6 @@ import io.jaegertracing.api_v2.JaegerSpanInternalModel.Span; import java.io.File; import java.nio.file.Path; -import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -86,6 +85,9 @@ public void testIngestionPacketFlow(@TempDir Path tempDir) { new JaegerSpanSerde().serializer()); spanNormalizerInputTopic.pipeInput(span); + // we use stream time for emit in grouper hence this additional pipe of a span after grouping + // window + spanNormalizerInputTopic.pipeInput(null, span, System.currentTimeMillis() + 30_001); // create output topic for span-normalizer topology TestOutputTopic spanNormalizerOutputTopic = @@ -96,8 +98,6 @@ public void testIngestionPacketFlow(@TempDir Path tempDir) { new AvroSerde<>().deserializer()); assertNotNull(spanNormalizerOutputTopic.readKeyValue()); - topologyTestDriver.advanceWallClockTime(Duration.ofSeconds(32)); - // create output topic for span-grouper topology TestOutputTopic spanGrouperOutputTopic = topologyTestDriver.createOutputTopic( From 821a50c72982035d673a7fd00406c052fd05b659 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 20:11:33 +0530 Subject: [PATCH 18/31] suppressions --- owasp-suppressions.xml | 21 +++------------------ 1 file changed, 3 insertions(+), 18 deletions(-) diff --git a/owasp-suppressions.xml b/owasp-suppressions.xml index 509d87f40..30f80369a 100644 --- a/owasp-suppressions.xml +++ b/owasp-suppressions.xml @@ -60,21 +60,6 @@ ^pkg:maven/io\.opentelemetry/opentelemetry\-exporter\-prometheus@.*$ cpe:/a:prometheus:prometheus - - - ^pkg:maven/org\.json/json@.*$ - CVE-2022-45688 - - - - ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ - CVE-2023-35116 - ^pkg:maven/org\.hypertrace\.core\.kafkastreams\.framework/avro\-partitioners@.*$ CVE-2023-37475 - + ^pkg:maven/org\.quartz\-scheduler/quartz@.*$ CVE-2023-39017 - + @@ -104,7 +89,7 @@ ^pkg:maven/com\.fasterxml\.jackson\.core/jackson\-databind@.*$ CVE-2023-35116 - + From 823369d4fcbda41e7b06fa40171ab1eb069b84ed Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 20:15:01 +0530 Subject: [PATCH 19/31] update test --- .github/workflows/hypertrace-ingester/scripts/tests.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.github/workflows/hypertrace-ingester/scripts/tests.sh b/.github/workflows/hypertrace-ingester/scripts/tests.sh index b271c9f2e..c4a8f4f3a 100755 --- a/.github/workflows/hypertrace-ingester/scripts/tests.sh +++ b/.github/workflows/hypertrace-ingester/scripts/tests.sh @@ -38,6 +38,10 @@ echo "Calling the frontend to generate a trace..." echo "" curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } +# grouper depends on stream time advancement, hence we are sending another set of traces just to meet that condition +sleep 35 +curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } + echo "Retrieving the list of traces." echo "" From 42b4d34a7c3c4f7ab08fc02920f1c8bef43ca16a Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 20:21:32 +0530 Subject: [PATCH 20/31] netty --- owasp-suppressions.xml | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/owasp-suppressions.xml b/owasp-suppressions.xml index 30f80369a..3432d98a0 100644 --- a/owasp-suppressions.xml +++ b/owasp-suppressions.xml @@ -91,16 +91,14 @@ - ^pkg:maven/io\.netty/netty\-handler@.*$ - CVE-2023-4586 - - - - ^pkg:maven/io\.netty/netty\-handler@.*$ + ^pkg:maven/io\.netty/netty.*@.*$ CVE-2023-4586 \ No newline at end of file From fc8d596d6a9d97be9e7762d1b77ea7d3d4ff0df7 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 20:30:53 +0530 Subject: [PATCH 21/31] update --- .github/workflows/hypertrace-ingester/scripts/tests.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/hypertrace-ingester/scripts/tests.sh b/.github/workflows/hypertrace-ingester/scripts/tests.sh index c4a8f4f3a..cc7f1c69d 100755 --- a/.github/workflows/hypertrace-ingester/scripts/tests.sh +++ b/.github/workflows/hypertrace-ingester/scripts/tests.sh @@ -39,7 +39,7 @@ echo "" curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } # grouper depends on stream time advancement, hence we are sending another set of traces just to meet that condition -sleep 35 +sleep 60 curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } echo "Retrieving the list of traces." From c3171ed9437db19fee8db38856957c1cf49ed46f Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 20:58:37 +0530 Subject: [PATCH 22/31] update --- .github/workflows/hypertrace-ingester/scripts/tests.sh | 2 +- .../src/main/resources/configs/common/application.conf | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hypertrace-ingester/scripts/tests.sh b/.github/workflows/hypertrace-ingester/scripts/tests.sh index cc7f1c69d..c4a8f4f3a 100755 --- a/.github/workflows/hypertrace-ingester/scripts/tests.sh +++ b/.github/workflows/hypertrace-ingester/scripts/tests.sh @@ -39,7 +39,7 @@ echo "" curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } # grouper depends on stream time advancement, hence we are sending another set of traces just to meet that condition -sleep 60 +sleep 35 curl -o /dev/null -s http://${FRONTEND_SERVICE_HOST}:8081 || { echo "Host $FRONTEND_SERVICE_HOST is down." ; exit 1; } echo "Retrieving the list of traces." diff --git a/hypertrace-trace-enricher/hypertrace-trace-enricher/src/main/resources/configs/common/application.conf b/hypertrace-trace-enricher/hypertrace-trace-enricher/src/main/resources/configs/common/application.conf index f328b9cfd..2be70f962 100644 --- a/hypertrace-trace-enricher/hypertrace-trace-enricher/src/main/resources/configs/common/application.conf +++ b/hypertrace-trace-enricher/hypertrace-trace-enricher/src/main/resources/configs/common/application.conf @@ -114,7 +114,7 @@ enricher { } TraceStatsEnricher { - class = "org.hypertrace.traceenricher.enrichment.enrichers.TraceStatsEnrichere" + class = "org.hypertrace.traceenricher.enrichment.enrichers.TraceStatsEnricher" dependencies = ["EndpointEnricher"] } From 89792480614046325226431041aa67793ec8281f Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 21:07:43 +0530 Subject: [PATCH 23/31] test --- .github/workflows/hypertrace-ingester/docker-compose.yml | 2 +- .../workflows/hypertrace-ingester/postgres/docker-compose.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hypertrace-ingester/docker-compose.yml b/.github/workflows/hypertrace-ingester/docker-compose.yml index 841eb08fb..a013d7c73 100644 --- a/.github/workflows/hypertrace-ingester/docker-compose.yml +++ b/.github/workflows/hypertrace-ingester/docker-compose.yml @@ -43,7 +43,7 @@ services: condition: service_healthy sut: - image: hypertrace/hypertrace-ingester:main + image: hypertrace/hypertrace-ingester:test container_name: sut environment: - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 diff --git a/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml b/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml index ba7450226..54f763876 100644 --- a/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml +++ b/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml @@ -49,7 +49,7 @@ services: condition: service_healthy # all-in-one ingestion pipeline for hypertrace sut: - image: hypertrace/hypertrace-ingester:main + image: hypertrace/hypertrace-ingester:test container_name: sut environment: - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 From acd16c1b93641ac500b2856154a559d887f1359a Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Fri, 13 Oct 2023 21:09:18 +0530 Subject: [PATCH 24/31] revert --- .github/workflows/hypertrace-ingester/docker-compose.yml | 2 +- .../workflows/hypertrace-ingester/postgres/docker-compose.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/hypertrace-ingester/docker-compose.yml b/.github/workflows/hypertrace-ingester/docker-compose.yml index a013d7c73..841eb08fb 100644 --- a/.github/workflows/hypertrace-ingester/docker-compose.yml +++ b/.github/workflows/hypertrace-ingester/docker-compose.yml @@ -43,7 +43,7 @@ services: condition: service_healthy sut: - image: hypertrace/hypertrace-ingester:test + image: hypertrace/hypertrace-ingester:main container_name: sut environment: - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 diff --git a/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml b/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml index 54f763876..ba7450226 100644 --- a/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml +++ b/.github/workflows/hypertrace-ingester/postgres/docker-compose.yml @@ -49,7 +49,7 @@ services: condition: service_healthy # all-in-one ingestion pipeline for hypertrace sut: - image: hypertrace/hypertrace-ingester:test + image: hypertrace/hypertrace-ingester:main container_name: sut environment: - KAFKA_BOOTSTRAP_SERVERS=kafka:9092 From 5f1b48c34884f6ebd7c3284617fee6f0e7a4da22 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 11:20:05 +0530 Subject: [PATCH 25/31] update --- .../src/main/resources/configs/common/application.conf | 1 + 1 file changed, 1 insertion(+) 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 64f47ac24..1849d5e97 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 @@ -18,6 +18,7 @@ kafka.streams.config = { replication.factor = 3 replication.factor = ${?REPLICATION_FACTOR} topic.cleanup.policy = "delete,compact" + consumer.session.timeout.ms = 300000 producer.max.request.size = 10485760 default.production.exception.handler = org.hypertrace.core.kafkastreams.framework.exceptionhandlers.IgnoreProductionExceptionHandler From 3e76ca0d02001f93b19070149128e008bfa2b8cf Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 11:31:47 +0530 Subject: [PATCH 26/31] log --- .github/workflows/docker-tests.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml index ecb2c0027..2854c7f40 100644 --- a/.github/workflows/docker-tests.yml +++ b/.github/workflows/docker-tests.yml @@ -41,6 +41,7 @@ jobs: run: | sleep 60 # you can decrease it but never increase it docker-compose -f docker-compose.yml ps + ../scripts/inspect.sh - name: Runs tests working-directory: ./.github/workflows/hypertrace-ingester/scripts From ec19f9720469b1c2860b48b8c78e57d4ab46b94c Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 12:16:57 +0530 Subject: [PATCH 27/31] wf changes --- .github/workflows/docker-tests.yml | 7 ++++--- .github/workflows/hypertrace-ingester/scripts/inspect.sh | 6 +----- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml index 2854c7f40..9231b5106 100644 --- a/.github/workflows/docker-tests.yml +++ b/.github/workflows/docker-tests.yml @@ -34,14 +34,14 @@ jobs: working-directory: ./.github/workflows/hypertrace-ingester # Below tests a docker-compose.yml service named 'sut' with a valid HEALTHCHECK instruction: run: | - docker-compose -f docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ../scripts/inspect.sh ; exit 1 ; } + docker-compose -f docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml ; exit 1 ; } - name: Waits for some stability working-directory: ./.github/workflows/hypertrace-ingester run: | sleep 60 # you can decrease it but never increase it docker-compose -f docker-compose.yml ps - ../scripts/inspect.sh + ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml - name: Runs tests working-directory: ./.github/workflows/hypertrace-ingester/scripts @@ -68,13 +68,14 @@ jobs: working-directory: ./.github/workflows/hypertrace-ingester # Below tests a docker-compose.yml service named 'sut' with a valid HEALTHCHECK instruction: run: | - docker-compose -f postgres/docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ../scripts/inspect.sh ; exit 1 ; } + docker-compose -f postgres/docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml ; exit 1 ; } - name: Waits for some stability working-directory: ./.github/workflows/hypertrace-ingester run: | sleep 60 # you can decrease it but never increase it docker-compose -f postgres/docker-compose.yml ps + ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml - name: Runs tests working-directory: ./.github/workflows/hypertrace-ingester/scripts diff --git a/.github/workflows/hypertrace-ingester/scripts/inspect.sh b/.github/workflows/hypertrace-ingester/scripts/inspect.sh index ec16206c1..a0255d3cd 100755 --- a/.github/workflows/hypertrace-ingester/scripts/inspect.sh +++ b/.github/workflows/hypertrace-ingester/scripts/inspect.sh @@ -1,11 +1,7 @@ #!/bin/bash -# This script displays the state and logs for the containers in the docker-compose. -SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" -DOCKER_COMPOSE_FILE_DIR="$(dirname $SCRIPT_DIR)/docker" - -containers=$(docker-compose -f ${DOCKER_COMPOSE_FILE_DIR}/docker-compose.yml -f ${DOCKER_COMPOSE_FILE_DIR}/docker-compose-zipkin-example.yml ps -q -a) +containers=$(docker-compose -f $1 -f $2 ps -q -a) while IFS= read -r container; do name=$(docker inspect $container | jq -r '.[0].Name') echo "==================" From 159ca1639403858904ec32cb6c237aaced9dca5b Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 12:23:34 +0530 Subject: [PATCH 28/31] inspect --- .github/workflows/hypertrace-ingester/scripts/inspect.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/hypertrace-ingester/scripts/inspect.sh b/.github/workflows/hypertrace-ingester/scripts/inspect.sh index a0255d3cd..5e86e330e 100755 --- a/.github/workflows/hypertrace-ingester/scripts/inspect.sh +++ b/.github/workflows/hypertrace-ingester/scripts/inspect.sh @@ -1,5 +1,6 @@ #!/bin/bash +echo "Inspecting compose containers from $1 and $2" containers=$(docker-compose -f $1 -f $2 ps -q -a) while IFS= read -r container; do From b8fd337b65ea82e1e180736859eaff62a17186a2 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 12:29:12 +0530 Subject: [PATCH 29/31] wf test --- .github/workflows/docker-tests.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml index 9231b5106..bc22bedaa 100644 --- a/.github/workflows/docker-tests.yml +++ b/.github/workflows/docker-tests.yml @@ -5,9 +5,10 @@ on: - main paths-ignore: - '**/*.md' - pull_request_target: - branches: - - main + # TODO: commented for purpose of testing wf changes, undo +# pull_request_target: +# branches: +# - main # workflow_dispatch will let us manually trigger the workflow from GitHub actions dashboard. # See https://docs.github.com/en/free-pro-team@latest/actions/managing-workflow-runs/manually-running-a-workflow workflow_dispatch: From 0494b5758d2da1f18810e48a837dd1d963c6028e Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 12:33:59 +0530 Subject: [PATCH 30/31] Revert "wf test" This reverts commit b8fd337b65ea82e1e180736859eaff62a17186a2. --- .github/workflows/docker-tests.yml | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml index bc22bedaa..9231b5106 100644 --- a/.github/workflows/docker-tests.yml +++ b/.github/workflows/docker-tests.yml @@ -5,10 +5,9 @@ on: - main paths-ignore: - '**/*.md' - # TODO: commented for purpose of testing wf changes, undo -# pull_request_target: -# branches: -# - main + pull_request_target: + branches: + - main # workflow_dispatch will let us manually trigger the workflow from GitHub actions dashboard. # See https://docs.github.com/en/free-pro-team@latest/actions/managing-workflow-runs/manually-running-a-workflow workflow_dispatch: From d21c403fcbb233abe82169b5083daccd4dfd99c8 Mon Sep 17 00:00:00 2001 From: kishansairam9 Date: Tue, 17 Oct 2023 12:34:53 +0530 Subject: [PATCH 31/31] fix --- .github/workflows/docker-tests.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/docker-tests.yml b/.github/workflows/docker-tests.yml index 9231b5106..e3bd80e14 100644 --- a/.github/workflows/docker-tests.yml +++ b/.github/workflows/docker-tests.yml @@ -68,14 +68,14 @@ jobs: working-directory: ./.github/workflows/hypertrace-ingester # Below tests a docker-compose.yml service named 'sut' with a valid HEALTHCHECK instruction: run: | - docker-compose -f postgres/docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml ; exit 1 ; } + docker-compose -f postgres/docker-compose.yml -f docker-compose-zipkin-example.yml up -d || { ./scripts/inspect.sh postgres/docker-compose.yml docker-compose-zipkin-example.yml ; exit 1 ; } - name: Waits for some stability working-directory: ./.github/workflows/hypertrace-ingester run: | sleep 60 # you can decrease it but never increase it docker-compose -f postgres/docker-compose.yml ps - ./scripts/inspect.sh docker-compose.yml docker-compose-zipkin-example.yml + ./scripts/inspect.sh postgres/docker-compose.yml docker-compose-zipkin-example.yml - name: Runs tests working-directory: ./.github/workflows/hypertrace-ingester/scripts