-
Notifications
You must be signed in to change notification settings - Fork 16
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
* Migrate span-normalizer to kafka streams * use trace_id as key when sending RawSpans to the output topic. The raw-spans-grouper can then do a groupByKey on the trace_id * Check if KStream for an input topic already exists and re-use it. * Extract common streams config * dd producer.max.request.size property * merged topic creationg changes as part of this PR * Schema registry configurable and decoupled * 1. bring in changes from job-config branch 2. clean up flink dependencies 3. add a simple integration test * revert config changes done for local testing * remove dev docker registry used for testing * Update span-normalizer/build.gradle.kts Co-authored-by: Ronak <[email protected]> Co-authored-by: Laxman Ch <[email protected]> Co-authored-by: kotharironak <[email protected]>
- Loading branch information
1 parent
82dcfdc
commit e4841cb
Showing
15 changed files
with
359 additions
and
370 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
80 changes: 72 additions & 8 deletions
80
span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/SpanNormalizer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,26 +1,90 @@ | ||
package org.hypertrace.core.spannormalizer; | ||
|
||
import static org.hypertrace.core.spannormalizer.constants.SpanNormalizerConstants.INPUT_TOPIC_CONFIG_KEY; | ||
import static org.hypertrace.core.spannormalizer.constants.SpanNormalizerConstants.KAFKA_STREAMS_CONFIG_KEY; | ||
import static org.hypertrace.core.spannormalizer.constants.SpanNormalizerConstants.OUTPUT_TOPIC_CONFIG_KEY; | ||
import static org.hypertrace.core.spannormalizer.constants.SpanNormalizerConstants.SPAN_NORMALIZER_JOB_CONFIG; | ||
|
||
import com.typesafe.config.Config; | ||
import org.hypertrace.core.serviceframework.background.PlatformBackgroundJob; | ||
import org.hypertrace.core.serviceframework.background.PlatformBackgroundService; | ||
import io.jaegertracing.api_v2.JaegerSpanInternalModel.Span; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import org.apache.kafka.common.serialization.Serdes; | ||
import org.apache.kafka.streams.StreamsBuilder; | ||
import org.apache.kafka.streams.kstream.Consumed; | ||
import org.apache.kafka.streams.kstream.KStream; | ||
import org.apache.kafka.streams.kstream.Produced; | ||
import org.hypertrace.core.kafkastreams.framework.KafkaStreamsApp; | ||
import org.hypertrace.core.serviceframework.config.ConfigClient; | ||
import org.hypertrace.core.serviceframework.config.ConfigUtils; | ||
import org.hypertrace.core.spannormalizer.jaeger.JaegerSpanSerde; | ||
import org.hypertrace.core.spannormalizer.jaeger.JaegerSpanToAvroRawSpanTransformer; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class SpanNormalizer extends PlatformBackgroundService { | ||
private static final Logger LOGGER = LoggerFactory.getLogger(SpanNormalizer.class); | ||
public class SpanNormalizer extends KafkaStreamsApp { | ||
|
||
private static final Logger logger = LoggerFactory.getLogger(SpanNormalizer.class); | ||
|
||
public SpanNormalizer(ConfigClient configClient) { | ||
super(configClient); | ||
} | ||
|
||
@Override | ||
protected PlatformBackgroundJob createBackgroundJob(Config config) { | ||
return new SpanNormalizerJob(config); | ||
public StreamsBuilder buildTopology(Map<String, Object> streamsProperties, | ||
StreamsBuilder streamsBuilder, | ||
Map<String, KStream<?, ?>> inputStreams) { | ||
|
||
Config jobConfig = getJobConfig(streamsProperties); | ||
String inputTopic = jobConfig.getString(INPUT_TOPIC_CONFIG_KEY); | ||
String outputTopic = jobConfig.getString(OUTPUT_TOPIC_CONFIG_KEY); | ||
|
||
KStream<byte[], Span> inputStream = (KStream<byte[], Span>) inputStreams.get(inputTopic); | ||
if (inputStream == null) { | ||
inputStream = streamsBuilder | ||
.stream(inputTopic, Consumed.with(Serdes.ByteArray(), new JaegerSpanSerde())); | ||
inputStreams.put(inputTopic, inputStream); | ||
} | ||
|
||
inputStream | ||
.transform(JaegerSpanToAvroRawSpanTransformer::new) | ||
.to(outputTopic, Produced.keySerde(Serdes.String())); | ||
|
||
return streamsBuilder; | ||
} | ||
|
||
@Override | ||
public Map<String, Object> getStreamsConfig(Config jobConfig) { | ||
Map<String, Object> streamsConfig = new HashMap<>( | ||
ConfigUtils.getFlatMapConfig(jobConfig, KAFKA_STREAMS_CONFIG_KEY)); | ||
return streamsConfig; | ||
} | ||
|
||
@Override | ||
public String getJobConfigKey() { | ||
return SPAN_NORMALIZER_JOB_CONFIG; | ||
} | ||
|
||
@Override | ||
public Logger getLogger() { | ||
return logger; | ||
} | ||
|
||
@Override | ||
protected Logger getLogger() { | ||
return LOGGER; | ||
public List<String> getInputTopics(Map<String, Object> properties) { | ||
Config jobConfig = getJobConfig(properties); | ||
return Collections.singletonList(jobConfig.getString(INPUT_TOPIC_CONFIG_KEY)); | ||
} | ||
|
||
@Override | ||
public List<String> getOutputTopics(Map<String, Object> properties) { | ||
Config jobConfig = getJobConfig(properties); | ||
return Collections.singletonList(jobConfig.getString(OUTPUT_TOPIC_CONFIG_KEY)); | ||
} | ||
|
||
private Config getJobConfig(Map<String, Object> properties) { | ||
return (Config) properties.get(getJobConfigKey()); | ||
} | ||
} |
132 changes: 0 additions & 132 deletions
132
span-normalizer/src/main/java/org/hypertrace/core/spannormalizer/SpanNormalizerJob.java
This file was deleted.
Oops, something went wrong.
14 changes: 14 additions & 0 deletions
14
...r/src/main/java/org/hypertrace/core/spannormalizer/constants/SpanNormalizerConstants.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,14 @@ | ||
package org.hypertrace.core.spannormalizer.constants; | ||
|
||
public class SpanNormalizerConstants { | ||
|
||
public static final String SCHEMA_REGISTRY_CONFIG_KEY = "schema.registry.config"; | ||
public static final String KAFKA_CONFIG_KEY = "kafka.config"; | ||
public static final String KAFKA_STREAMS_CONFIG_KEY = "kafka.streams.config"; | ||
public static final String JOB_CONFIG = "jobconfig"; | ||
public static final String SPAN_NORMALIZER_CONFIG_KEY = "span.normalizer.config"; | ||
public static final String INPUT_TOPIC_CONFIG_KEY = "input.topic"; | ||
public static final String OUTPUT_TOPIC_CONFIG_KEY = "output.topic"; | ||
public static final String SPAN_TYPE_CONFIG_KEY = "span.type"; | ||
public static final String SPAN_NORMALIZER_JOB_CONFIG = "span-normalizer-job-config"; | ||
} |
30 changes: 0 additions & 30 deletions
30
...-normalizer/src/main/java/org/hypertrace/core/spannormalizer/jaeger/JaegerSpanSchema.java
This file was deleted.
Oops, something went wrong.
Oops, something went wrong.