diff --git a/.gitignore b/.gitignore
index 7f219aa..4708ed8 100644
--- a/.gitignore
+++ b/.gitignore
@@ -8,4 +8,5 @@ target/
*/.project
*/.settings
*/.classpath
-*/.factorypath
\ No newline at end of file
+*/.factorypath
+*.DS_Store
\ No newline at end of file
diff --git a/Makefile b/Makefile
index 880d30e..c70903e 100644
--- a/Makefile
+++ b/Makefile
@@ -1,4 +1,4 @@
-.PHONY: all clean release json-transformer kafka-producer http-poster firehose-writer secret-detector span-key-extractor
+.PHONY: all clean release json-transformer kafka-producer http-poster firehose-writer secret-detector span-key-extractor sample-key-extractor
PWD := $(shell pwd)
@@ -28,11 +28,15 @@ secret-detector:
span-key-extractor:
mvn package -DfinalName=span-key-extractor -pl span-key-extractor -am
+sample-key-extractor:
+ mvn package -DfinalName=sample-key-extractor -pl sample-key-extractor -am
+
# build all and release
-release: clean span-key-extractor json-transformer kafka-producer http-poster firehose-writer secret-detector
+release: clean span-key-extractor json-transformer kafka-producer http-poster firehose-writer secret-detector sample-key-extractor
cd json-transformer && $(MAKE) release
cd kafka-producer && $(MAKE) release
cd http-poster && $(MAKE) release
cd firehose-writer && $(MAKE) release
cd secret-detector && $(MAKE) release
+ cd sample-key-extractor && $(MAKE) release
./.travis/deploy.sh
diff --git a/commons/pom.xml b/commons/pom.xml
index 5a6d595..826527e 100644
--- a/commons/pom.xml
+++ b/commons/pom.xml
@@ -8,7 +8,7 @@
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
diff --git a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaConsumerStarter.java b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaConsumerStarter.java
index 43998d6..992496c 100644
--- a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaConsumerStarter.java
+++ b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaConsumerStarter.java
@@ -42,19 +42,22 @@ public class KafkaConsumerStarter {
public final Class> containingClass;
public final String clientId;
private final List tasks;
+ private final KafkaConfig kafkaConfig;
public KafkaConsumerStarter(Class> containingClass,
String clientId,
+ KafkaConfig kafkaConfig,
HealthController healthController) {
this.containingClass = containingClass;
this.clientId = clientId;
+ this.kafkaConfig = kafkaConfig;
this.healthController = healthController;
this.tasks = new ArrayList<>();
}
public void createAndStartConsumer(SpanProcessorSupplier processorSupplier) {
for(int idx = 0; idx <= getThreadCount(); idx++) {
- final ConsumerTask task = new ConsumerTask(getKafkaConfig(), containingClass, processorSupplier, healthController);
+ final ConsumerTask task = new ConsumerTask(kafkaConfig, containingClass, processorSupplier, healthController);
this.tasks.add(task);
final Thread thread = new Thread(task);
thread.setDaemon(true);
@@ -74,11 +77,6 @@ private void close() {
}
private int getThreadCount() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.threadcount();
}
-
- private static KafkaConfig getKafkaConfig() {
- return CONFIGURATION_PROVIDER.bind(HAYSTACK_KAFKA_CONFIG_PREFIX, KafkaConfig.class);
- }
}
diff --git a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamBuilderBase.java b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamBuilderBase.java
index 6ad1b65..d66a4c9 100644
--- a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamBuilderBase.java
+++ b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamBuilderBase.java
@@ -13,16 +13,16 @@ public abstract class KafkaStreamBuilderBase implements KafkaStreamBuilder, Main
private final KafkaStreamStarter kafkaStreamStarter;
private final SerdeFactory serdeFactory;
private final String application;
- private final KafkaConfigurationProvider kafkaConfigurationProvider;
+ private final KafkaConfig kafkaConfig;
private final ForeachAction foreachAction;
private final ProcessorSupplier processorSupplier;
public KafkaStreamBuilderBase(KafkaStreamStarter kafkaStreamStarter,
SerdeFactory serdeFactory,
String application,
- KafkaConfigurationProvider kafkaConfigurationProvider,
+ KafkaConfig kafkaConfig,
ForeachAction foreachAction) {
- this(kafkaStreamStarter, serdeFactory, application, kafkaConfigurationProvider, foreachAction, null);
+ this(kafkaStreamStarter, serdeFactory, application, kafkaConfig, foreachAction, null);
}
@@ -37,13 +37,13 @@ public KafkaStreamBuilderBase(KafkaStreamStarter kafkaStreamStarter,
private KafkaStreamBuilderBase(KafkaStreamStarter kafkaStreamStarter,
SerdeFactory serdeFactory,
String application,
- KafkaConfigurationProvider kafkaConfigurationProvider,
+ KafkaConfig kafkaConfig,
ForeachAction foreachAction,
ProcessorSupplier processorSupplier) {
this.kafkaStreamStarter = kafkaStreamStarter;
this.serdeFactory = serdeFactory;
this.application = application;
- this.kafkaConfigurationProvider = kafkaConfigurationProvider;
+ this.kafkaConfig = kafkaConfig;
this.foreachAction = foreachAction;
this.processorSupplier = processorSupplier;
}
@@ -52,7 +52,7 @@ private KafkaStreamBuilderBase(KafkaStreamStarter kafkaStreamStarter,
public void buildStreamTopology(KStreamBuilder kStreamBuilder) {
final Serde stringSerde = Serdes.String();
final Serde spanSerde = serdeFactory.createJsonProtoSpanSerde(application);
- final String fromTopic = kafkaConfigurationProvider.fromtopic();
+ final String fromTopic = kafkaConfig.fromtopic();
final KStream stream = kStreamBuilder.stream(stringSerde, spanSerde, fromTopic);
if(foreachAction != null) {
stream.foreach(foreachAction);
diff --git a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarter.java b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarter.java
index 3973761..7b6f356 100644
--- a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarter.java
+++ b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarter.java
@@ -17,7 +17,6 @@
package com.expedia.www.haystack.pipes.commons.kafka;
import com.expedia.www.haystack.commons.config.Configuration;
-import com.expedia.www.haystack.pipes.commons.IntermediateStreamsConfig;
import com.expedia.www.haystack.pipes.commons.SystemExitUncaughtExceptionHandler;
import com.expedia.www.haystack.pipes.commons.health.HealthController;
import com.netflix.servo.util.VisibleForTesting;
@@ -32,8 +31,6 @@
import java.util.Properties;
-import static com.expedia.www.haystack.pipes.commons.Configuration.HAYSTACK_KAFKA_CONFIG_PREFIX;
-import static com.expedia.www.haystack.pipes.commons.Configuration.HAYSTACK_PIPE_STREAMS;
import static java.util.concurrent.TimeUnit.SECONDS;
public class KafkaStreamStarter {
@@ -58,13 +55,17 @@ public class KafkaStreamStarter {
public final Class extends KafkaStreamBuilder> containingClass;
public final String clientId;
private final StreamsConfig streamsConfig;
+ @VisibleForTesting
+ static KafkaConfig kafkaConfig;
public KafkaStreamStarter(Class extends KafkaStreamBuilder> containingClass,
String clientId,
+ KafkaConfig kafkaConfig,
HealthController healthController) {
this.containingClass = containingClass;
this.clientId = clientId;
this.healthController = healthController;
+ this.kafkaConfig = kafkaConfig;
this.streamsConfig = new StreamsConfig(getProperties());
}
@@ -97,47 +98,31 @@ Properties getProperties() {
props.put(ConsumerConfig.GROUP_ID_CONFIG, containingClass.getName());
props.put(StreamsConfig.APPLICATION_ID_CONFIG, containingClass.getSimpleName());
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, getIpAnPort());
- props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, getReplicationFactor());
props.put(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), getConsumerSessionTimeout());
props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, getThreadCount());
return props;
}
private String getIpAnPort() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.brokers() + ":" + kafkaConfig.port();
}
private String getFromTopic() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.fromtopic();
}
private String getToTopic() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.totopic();
}
private int getThreadCount() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.threadcount();
}
- private int getReplicationFactor() {
- final IntermediateStreamsConfig intermediateStreamsConfig = CONFIGURATION_PROVIDER.bind(
- HAYSTACK_PIPE_STREAMS, IntermediateStreamsConfig.class);
- return intermediateStreamsConfig.replicationfactor();
- }
-
private int getConsumerSessionTimeout() {
- final KafkaConfig kafkaConfig = getKafkaConfig();
return kafkaConfig.sessiontimeout();
}
- private static KafkaConfig getKafkaConfig() {
- return CONFIGURATION_PROVIDER.bind(HAYSTACK_KAFKA_CONFIG_PREFIX, KafkaConfig.class);
- }
-
static class Factory {
KStreamBuilder createKStreamBuilder() {
return new KStreamBuilder();
diff --git a/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/config/KafkaConsumerConfig.java b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/config/KafkaConsumerConfig.java
new file mode 100644
index 0000000..999b21b
--- /dev/null
+++ b/commons/src/main/java/com/expedia/www/haystack/pipes/commons/kafka/config/KafkaConsumerConfig.java
@@ -0,0 +1,90 @@
+package com.expedia.www.haystack.pipes.commons.kafka.config;
+
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfig;
+
+public class KafkaConsumerConfig implements KafkaConfig {
+
+ private String brokers;
+
+ private int port;
+
+ private String fromTopic;
+
+ private String toTopic;
+
+ private int threadCount;
+
+ private int sessionTimeout;
+
+ private int maxWakeUps;
+
+ private int wakeUpTimeoutMs;
+
+ private long pollTimeoutMs;
+
+ private long commitMs;
+
+ public KafkaConsumerConfig(final String brokers, final int port, final String fromTopic, final String toTopic, final int threadCount, final int sessionTimeout, final int maxWakeUps, final int wakeUpTimeoutMs, final long pollTimeoutMs, final long commitMs) {
+ this.brokers = brokers;
+ this.port = port;
+ this.fromTopic = fromTopic;
+ this.toTopic = toTopic;
+ this.threadCount = threadCount;
+ this.sessionTimeout = sessionTimeout;
+ this.maxWakeUps = maxWakeUps;
+ this.wakeUpTimeoutMs = wakeUpTimeoutMs;
+ this.pollTimeoutMs = pollTimeoutMs;
+ this.commitMs = commitMs;
+ }
+
+ @Override
+ public String brokers() {
+ return this.brokers;
+ }
+
+ @Override
+ public int port() {
+ return this.port;
+ }
+
+ @Override
+ public String fromtopic() {
+ return this.fromTopic;
+ }
+
+ @Override
+ public String totopic() {
+ return this.toTopic;
+ }
+
+ @Override
+ public int threadcount() {
+ return this.threadCount;
+ }
+
+ @Override
+ public int sessiontimeout() {
+ return this.sessionTimeout;
+ }
+
+ @Override
+ public int maxwakeups() {
+ return this.maxWakeUps;
+ }
+
+ @Override
+ public int wakeuptimeoutms() {
+ return this.wakeUpTimeoutMs;
+ }
+
+ @Override
+ public long polltimeoutms() {
+ return this.pollTimeoutMs;
+ }
+
+ @Override
+ public long commitms() {
+ return this.commitMs;
+ }
+
+}
diff --git a/commons/src/test/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarterTest.java b/commons/src/test/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarterTest.java
index 00f2abf..eb20293 100644
--- a/commons/src/test/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarterTest.java
+++ b/commons/src/test/java/com/expedia/www/haystack/pipes/commons/kafka/KafkaStreamStarterTest.java
@@ -73,8 +73,6 @@ public class KafkaStreamStarterTest {
@Mock
private SystemExitUncaughtExceptionHandler mockSystemExitUncaughtExceptionHandler;
@Mock
- private ConfigurationProvider mockConfigurationProvider;
- @Mock
private KafkaConfig mockKafkaConfig;
private KafkaStreamStarter kafkaStreamStarter;
@@ -86,7 +84,8 @@ public void setUp() {
KafkaStreamStarter.factory = mockFactory;
realLogger = KafkaStreamStarter.logger;
KafkaStreamStarter.logger = mockLogger;
- kafkaStreamStarter = new KafkaStreamStarter(mockKafkaStreamBuilder.getClass(), CLIENT_ID, healthController);
+ KafkaConfigurationProvider configurationProvider = new KafkaConfigurationProvider();
+ kafkaStreamStarter = new KafkaStreamStarter(mockKafkaStreamBuilder.getClass(), CLIENT_ID,configurationProvider,healthController);
}
@After
@@ -97,7 +96,7 @@ public void tearDown() {
PollScheduler.getInstance().stop();
}
verifyNoMoreInteractions(mockFactory, mockLogger, mockKafkaStreamBuilder, mockKStreamBuilder,
- mockKafkaStreams, mockSystemExitUncaughtExceptionHandler, mockConfigurationProvider, mockKafkaConfig);
+ mockKafkaStreams, mockSystemExitUncaughtExceptionHandler, mockKafkaConfig);
}
@Test
@@ -113,19 +112,17 @@ public void testCreateAndStartStreamWithToTopic() {
@Test
public void testCreateAndStartStreamWithoutToTopic() {
commonWhensForCreateAndStartStream();
- when(mockConfigurationProvider.bind(HAYSTACK_KAFKA_CONFIG_PREFIX, KafkaConfig.class)).thenReturn(mockKafkaConfig);
when(mockKafkaConfig.fromtopic()).thenReturn(KAFKA_FROM_TOPIC);
when(mockKafkaConfig.brokers()).thenReturn(BROKERS);
when(mockKafkaConfig.port()).thenReturn(PORT);
- final ConfigurationProvider savedConfigurationProvider = KafkaStreamStarter.CONFIGURATION_PROVIDER;
- KafkaStreamStarter.CONFIGURATION_PROVIDER = mockConfigurationProvider;
+ final KafkaConfig savedKafkaConfig = KafkaStreamStarter.kafkaConfig;
+ KafkaStreamStarter.kafkaConfig = mockKafkaConfig;
kafkaStreamStarter.createAndStartStream(mockKafkaStreamBuilder);
- KafkaStreamStarter.CONFIGURATION_PROVIDER = savedConfigurationProvider;
+ kafkaStreamStarter.kafkaConfig = savedKafkaConfig;
commonVerifiesForCreateAndStartStream();
verify(mockLogger).info(String.format(STARTING_MSG_WITHOUT_TO_TOPIC, KAFKA_IP_AND_PORT, KAFKA_FROM_TOPIC));
- verify(mockConfigurationProvider, times(3)).bind(HAYSTACK_KAFKA_CONFIG_PREFIX, KafkaConfig.class);
verify(mockKafkaConfig).fromtopic();
verify(mockKafkaConfig).totopic();
verify(mockKafkaConfig).brokers();
@@ -157,7 +154,6 @@ public void testGetProperties() {
assertEquals(mockKafkaStreamBuilder.getClass().getName(), properties.remove(ConsumerConfig.GROUP_ID_CONFIG));
assertEquals(mockKafkaStreamBuilder.getClass().getSimpleName(), properties.remove(StreamsConfig.APPLICATION_ID_CONFIG));
assertEquals(KAFKA_IP_AND_PORT, properties.remove(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG));
- assertEquals(2147483645, properties.remove(StreamsConfig.REPLICATION_FACTOR_CONFIG));
assertEquals(THREAD_COUNT_CONFIGURATION_IN_TEST_BASE_DOT_YAML, properties.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG));
assertEquals(15000, properties.remove(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG)));
assertEquals("Properties should be empty but is: " + properties, 0, properties.size());
diff --git a/commons/src/test/resources/base.yaml b/commons/src/test/resources/base.yaml
index e200120..a3f2521 100644
--- a/commons/src/test/resources/base.yaml
+++ b/commons/src/test/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 2147483645
graphite:
prefix: "haystack.graphite.prefix"
host: "haystack.graphite.host"
diff --git a/firehose-writer/pom.xml b/firehose-writer/pom.xml
index 90907c5..5c237b3 100644
--- a/firehose-writer/pom.xml
+++ b/firehose-writer/pom.xml
@@ -6,14 +6,14 @@
haystack-pipes-firehose-writer
jar
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
haystack-pipes-firehose-writer
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
diff --git a/firehose-writer/src/main/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfig.java b/firehose-writer/src/main/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfig.java
index 5e0d61b..2585296 100644
--- a/firehose-writer/src/main/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfig.java
+++ b/firehose-writer/src/main/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfig.java
@@ -55,13 +55,20 @@
@Configuration
@ComponentScan(basePackageClasses = SpringConfig.class)
class SpringConfig {
- @VisibleForTesting static final String SPAN_COUNTER_NAME = "REQUEST";
- @VisibleForTesting static final String SUCCESS_COUNTER_NAME = "SUCCESS";
- @VisibleForTesting static final String FAILURE_COUNTER_NAME = "FAILURE";
- @VisibleForTesting static final String EXCEPTION_COUNTER_NAME = "EXCEPTION";
- @VisibleForTesting static final String THROTTLED_COUNTER_NAME = "THROTTLED";
- @VisibleForTesting static final String SOCKET_TIMEOUT_COUNTER_NAME = "SOCKET_TIMEOUT";
- @VisibleForTesting static final String PUT_BATCH_REQUEST_TIMER_NAME = "PUT_BATCH_REQUEST";
+ @VisibleForTesting
+ static final String SPAN_COUNTER_NAME = "REQUEST";
+ @VisibleForTesting
+ static final String SUCCESS_COUNTER_NAME = "SUCCESS";
+ @VisibleForTesting
+ static final String FAILURE_COUNTER_NAME = "FAILURE";
+ @VisibleForTesting
+ static final String EXCEPTION_COUNTER_NAME = "EXCEPTION";
+ @VisibleForTesting
+ static final String THROTTLED_COUNTER_NAME = "THROTTLED";
+ @VisibleForTesting
+ static final String SOCKET_TIMEOUT_COUNTER_NAME = "SOCKET_TIMEOUT";
+ @VisibleForTesting
+ static final String PUT_BATCH_REQUEST_TIMER_NAME = "PUT_BATCH_REQUEST";
private final MetricObjects metricObjects;
@@ -125,8 +132,8 @@ Timer spanArrivalTimer() {
@Bean
@Autowired
- KafkaConsumerStarter kafkaConsumerStarter(final HealthController healthController) {
- return new KafkaConsumerStarter(ProtobufToFirehoseProducer.class, APPLICATION, healthController);
+ KafkaConsumerStarter kafkaConsumerStarter(final HealthController healthController, KafkaConfigurationProvider kafkaConfigurationProvider) {
+ return new KafkaConsumerStarter(ProtobufToFirehoseProducer.class, APPLICATION, kafkaConfigurationProvider, healthController);
}
@Bean
@@ -266,7 +273,7 @@ S3Sender.Factory s3SenderFactory() {
}
@Bean
- Clock clock() {
+ Clock clock() {
return Clock.systemUTC();
}
@@ -280,12 +287,12 @@ Timers timers(Timer putBatchRequestTimer,
@Bean
@Autowired
FirehoseTimersAndCounters counters(Clock clock,
- Timers timers,
- Counter spanCounter,
- Counter successCounter,
- Counter failureCounter,
- Counter exceptionCounter,
- Counter socketTimeoutCounter) {
+ Timers timers,
+ Counter spanCounter,
+ Counter successCounter,
+ Counter failureCounter,
+ Counter exceptionCounter,
+ Counter socketTimeoutCounter) {
return new FirehoseTimersAndCounters(clock, timers,
spanCounter, successCounter, failureCounter, exceptionCounter, socketTimeoutCounter);
}
diff --git a/firehose-writer/src/main/resources/base.yaml b/firehose-writer/src/main/resources/base.yaml
index f026d21..ef0e6f9 100644
--- a/firehose-writer/src/main/resources/base.yaml
+++ b/firehose-writer/src/main/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
diff --git a/firehose-writer/src/test/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfigTest.java b/firehose-writer/src/test/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfigTest.java
index 9c6daa6..b31516e 100644
--- a/firehose-writer/src/test/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfigTest.java
+++ b/firehose-writer/src/test/java/com/expedia/www/haystack/pipes/firehoseWriter/SpringConfigTest.java
@@ -20,6 +20,7 @@
import com.amazonaws.client.builder.AwsClientBuilder.EndpointConfiguration;
import com.expedia.www.haystack.metrics.MetricObjects;
import com.expedia.www.haystack.pipes.commons.health.HealthController;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaConsumerStarter;
import com.netflix.servo.monitor.Counter;
import com.netflix.servo.monitor.Timer;
@@ -58,6 +59,8 @@ public class SpringConfigTest {
private Counter mockCounter;
@Mock
private HealthController mockHealthController;
+ @Mock
+ private KafkaConfigurationProvider mockKafkaConfigurationProvider;
private SpringConfig springConfig;
@@ -162,7 +165,7 @@ public void testSpanArrivalTimer() {
@Test
public void testKafkaStreamStarter() {
- final KafkaConsumerStarter kafkaStreamStarter = springConfig.kafkaConsumerStarter(mockHealthController);
+ final KafkaConsumerStarter kafkaStreamStarter = springConfig.kafkaConsumerStarter(mockHealthController, mockKafkaConfigurationProvider);
assertSame(ProtobufToFirehoseProducer.class, kafkaStreamStarter.containingClass);
assertSame(APPLICATION, kafkaStreamStarter.clientId);
}
diff --git a/http-poster/pom.xml b/http-poster/pom.xml
index 1ca88dc..bd176da 100644
--- a/http-poster/pom.xml
+++ b/http-poster/pom.xml
@@ -7,14 +7,14 @@
haystack-pipes-http-poster
jar
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
haystack-pipes-http-poster
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
diff --git a/http-poster/src/main/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfig.java b/http-poster/src/main/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfig.java
index 49de6a4..5795194 100644
--- a/http-poster/src/main/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfig.java
+++ b/http-poster/src/main/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfig.java
@@ -92,8 +92,8 @@ Timer spanArrivalTimer() {
@Bean
@Autowired
- KafkaStreamStarter kafkaStreamStarter(final HealthController healthController) {
- return new KafkaStreamStarter(ProtobufToHttpPoster.class, APPLICATION, healthController);
+ KafkaStreamStarter kafkaStreamStarter(final HealthController healthController, KafkaConfigurationProvider kafkaConfigurationProvider) {
+ return new KafkaStreamStarter(ProtobufToHttpPoster.class, APPLICATION, kafkaConfigurationProvider, healthController);
}
@Bean
diff --git a/http-poster/src/main/resources/base.yaml b/http-poster/src/main/resources/base.yaml
index b0fc0e7..56ec4ad 100644
--- a/http-poster/src/main/resources/base.yaml
+++ b/http-poster/src/main/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
diff --git a/http-poster/src/test/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfigTest.java b/http-poster/src/test/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfigTest.java
index 8abb131..0d435bf 100644
--- a/http-poster/src/test/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfigTest.java
+++ b/http-poster/src/test/java/com/expedia/www/haystack/pipes/httpPoster/SpringConfigTest.java
@@ -2,6 +2,7 @@
import com.expedia.www.haystack.metrics.MetricObjects;
import com.expedia.www.haystack.pipes.commons.health.HealthController;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
import org.junit.After;
import org.junit.Before;
@@ -29,6 +30,8 @@ public class SpringConfigTest {
private MetricObjects mockMetricObjects;
@Mock
private HealthController mockHealthController;
+ @Mock
+ private KafkaConfigurationProvider mockKafkaConfigurationProvider;
private SpringConfig springConfig;
@@ -76,7 +79,7 @@ public void testSpanArrivalTimer() {
@Test
public void testKafkaStreamStarter() {
- final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController);
+ final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController, mockKafkaConfigurationProvider);
assertSame(ProtobufToHttpPoster.class, kafkaStreamStarter.containingClass);
assertSame(APPLICATION, kafkaStreamStarter.clientId);
diff --git a/http-poster/src/test/resources/base.yaml b/http-poster/src/test/resources/base.yaml
index 43fdee2..8279e65 100644
--- a/http-poster/src/test/resources/base.yaml
+++ b/http-poster/src/test/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
diff --git a/json-transformer/pom.xml b/json-transformer/pom.xml
index eff45b2..41e9fe2 100644
--- a/json-transformer/pom.xml
+++ b/json-transformer/pom.xml
@@ -7,14 +7,14 @@
haystack-pipes-json-transformer
jar
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
haystack-pipes-json-transformer
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
diff --git a/json-transformer/src/main/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfig.java b/json-transformer/src/main/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfig.java
index ec164d4..738ad3a 100644
--- a/json-transformer/src/main/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfig.java
+++ b/json-transformer/src/main/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfig.java
@@ -2,6 +2,7 @@
import com.expedia.www.haystack.pipes.commons.health.HealthController;
import com.expedia.www.haystack.pipes.commons.health.UpdateHealthStatusFile;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
import org.slf4j.Logger;
@@ -22,10 +23,15 @@ Logger jsonTransformerIsActiveControllerLogger() {
return LoggerFactory.getLogger(JsonTransformerIsActiveController.class);
}
+ @Bean
+ KafkaConfigurationProvider kafkaConfigurationProvider() {
+ return new KafkaConfigurationProvider();
+ }
+
@Bean
@Autowired
- KafkaStreamStarter kafkaStreamStarter(final HealthController healthController) {
- return new KafkaStreamStarter(ProtobufToJsonTransformer.class, APPLICATION, healthController);
+ KafkaStreamStarter kafkaStreamStarter(final HealthController healthController, KafkaConfigurationProvider kafkaConfigurationProvider) {
+ return new KafkaStreamStarter(ProtobufToJsonTransformer.class, APPLICATION, kafkaConfigurationProvider, healthController);
}
@Bean
diff --git a/json-transformer/src/main/resources/base.yaml b/json-transformer/src/main/resources/base.yaml
index 6c9618c..a8057f1 100644
--- a/json-transformer/src/main/resources/base.yaml
+++ b/json-transformer/src/main/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
diff --git a/json-transformer/src/test/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfigTest.java b/json-transformer/src/test/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfigTest.java
index 39f5a60..c22b246 100644
--- a/json-transformer/src/test/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfigTest.java
+++ b/json-transformer/src/test/java/com/expedia/www/haystack/pipes/jsonTransformer/SpringConfigTest.java
@@ -1,7 +1,7 @@
package com.expedia.www.haystack.pipes.jsonTransformer;
import com.expedia.www.haystack.pipes.commons.health.HealthController;
-import com.expedia.www.haystack.pipes.commons.health.HealthStatusListener;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
import org.junit.After;
import org.junit.Before;
@@ -14,9 +14,7 @@
import static com.expedia.www.haystack.pipes.jsonTransformer.Constants.APPLICATION;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertSame;
-import static org.mockito.Matchers.any;
import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.verify;
@RunWith(MockitoJUnitRunner.class)
public class SpringConfigTest {
@@ -24,6 +22,9 @@ public class SpringConfigTest {
@Mock
private HealthController mockHealthController;
+ @Mock
+ private KafkaConfigurationProvider mockKafkaConfigurationProvider;
+
private SpringConfig springConfig;
@Before
@@ -45,7 +46,7 @@ public void testJsonTransformerIsActiveControllerLogger() {
@Test
public void testKafkaStreamStarter() {
- final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController);
+ final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController, mockKafkaConfigurationProvider);
assertSame(ProtobufToJsonTransformer.class, kafkaStreamStarter.containingClass);
assertSame(APPLICATION, kafkaStreamStarter.clientId);
diff --git a/json-transformer/src/test/resources/base.yaml b/json-transformer/src/test/resources/base.yaml
index cf7fbce..6fdf6f9 100644
--- a/json-transformer/src/test/resources/base.yaml
+++ b/json-transformer/src/test/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: 3000
polltimeoutms: 250
commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "localhost"
diff --git a/kafka-producer/README.md b/kafka-producer/README.md
index feadc36..cf62bfc 100644
--- a/kafka-producer/README.md
+++ b/kafka-producer/README.md
@@ -1,18 +1,34 @@
# kafka-producer
The `kafka-producer` service uses [Kafka Streams](https://kafka.apache.org/documentation/streams/) to read protobuf
-records from Kafka, transform them to tags-flattened JSON, and write the transformed record to another (typically
+records from Kafka, it uses SPI model to load key-extractor to transform span, and write the transformed record to another (typically
external) Kafka queue and topic. The code is simple and self-explanatory and consists of the following classes:
-1. A [Kafka producer](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducer.java)
+1. Span Key extractor is required for `kafka-producer` to work. The [sample extractor](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java)
+transforms span to tags-flattened JSON.
+2. A [Kafka producer](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducer.java)
that wires serializers and deserializers into a
[Kafka Streams](https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams) pipeline.
-2. A simple Spring Boot [application](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveController.java)
+3. A simple java [application](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Service.java)
that provides an HTTP endpoint, used for health checks.
-3. [Configurations](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfig.java)
-for the external (probably) Kafka queue to which the Spans' JSON is written.
-4. A [Kafka for/each action](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaAction.java)
+4. [Configurations](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/config/KafkaProducerConfig.java)
+for the external (probably)/another Kafka queue to which the Spans' JSON is written.
+5. A [Kafka for/each action](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipeline.java)
at the end the Kafka Streams pipeline that writes ("produces") to the external Kafka queue.
-5. [Unit tests](https://github.com/ExpediaDotCom/haystack-pipes/tree/master/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer)
+6. [Unit tests](https://github.com/ExpediaDotCom/haystack-pipes/tree/master/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer)
Various classes from the [commons](https://github.com/ExpediaDotCom/haystack-pipes/tree/master/commons)
package are also used.
+
+###Deployment
+ Docker image of this module can be made using [Dockerfile](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/build/docker/Dockerfile).
+ The above docker image is not the deploy-ready image. It would require Extractors provided by Java SPI model.
+
+ For eg: building docker image using [Dockerfile](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/sample-key-extractor/build/docker/Dockerfile),
+ would give you [Sample Extractor](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java).
+ with this module.The configurations for Sample Extractors are added in [base.conf](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/resources/config/base.conf).
+
+
+ You can load more than one extractors by adding code in the project and the file paths in [file](https://github.com/ExpediaDotCom/haystack-pipes/tree/master/sample-key-extractor/src/main/resources/META-INF/services)
+ and configurations required can be added in [base.conf](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/resources/config/base.conf).
+ Or we can override the conf file using environment variable as `HAYSTACK_OVERRIDES_CONFIG_PATH`.
+
\ No newline at end of file
diff --git a/kafka-producer/pom.xml b/kafka-producer/pom.xml
index 6007686..3dcf67e 100644
--- a/kafka-producer/pom.xml
+++ b/kafka-producer/pom.xml
@@ -6,14 +6,14 @@
haystack-pipes-kafka-producer
jar
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
haystack-pipes-kafka-producer
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
@@ -24,6 +24,7 @@
com.expedia.www
haystack-pipes-commons
+ ${project.version}
com.expedia.www
@@ -62,6 +63,19 @@
org.springframework
spring-context
+
+ com.expedia.www
+ haystack-commons
+
+
+ com.typesafe
+ config
+
+
+ com.expedia.www
+ span-key-extractor
+ ${project.version}
+
@@ -89,7 +103,7 @@
org.springframework.boot
spring-boot-maven-plugin
- com.expedia.www.haystack.pipes.kafkaProducer.KafkaProducerIsActiveController
+ com.expedia.www.haystack.pipes.kafka.producer.Service
diff --git a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Constants.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/CallbackFactory.java
similarity index 52%
rename from span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Constants.java
rename to kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/CallbackFactory.java
index 7294895..d4e0215 100644
--- a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Constants.java
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/CallbackFactory.java
@@ -14,12 +14,23 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.key.extractor;
+package com.expedia.www.haystack.pipes.kafka.producer;
-public interface Constants {
- String HAYSTACK_KEY = "haystack";
- String DIRECTORY_KEY = "directory";
- String EXTRACTORS_KEY = "extractors";
- String EXTRACTOR_NAME_KEY = "name";
- String EXTRACTOR_CONFIG_KEY = "config";
+import org.apache.commons.pool2.BasePooledObjectFactory;
+import org.apache.commons.pool2.PooledObject;
+import org.apache.commons.pool2.impl.DefaultPooledObject;
+
+public class CallbackFactory extends BasePooledObjectFactory {
+
+
+ @Override
+ public KafkaCallback create() {
+ return new KafkaCallback();
+ }
+
+ @Override
+ public PooledObject wrap(
+ KafkaCallback produceIntoExternalKafkaCallback) {
+ return new DefaultPooledObject<>(produceIntoExternalKafkaCallback);
+ }
}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/Constants.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Constants.java
similarity index 81%
rename from kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/Constants.java
rename to kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Constants.java
index e461bb7..3eb6b92 100644
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/Constants.java
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Constants.java
@@ -14,8 +14,8 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.kafkaProducer;
+package com.expedia.www.haystack.pipes.kafka.producer;
-public interface Constants {
- String APPLICATION = "haystack-pipes-kafka-producer";
+public final class Constants {
+ static String APPLICATION = "haystack-pipes-kafka-producer";
}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallback.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallback.java
similarity index 72%
rename from kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallback.java
rename to kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallback.java
index 8509767..74f1fa2 100644
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallback.java
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallback.java
@@ -14,30 +14,22 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.kafkaProducer;
+package com.expedia.www.haystack.pipes.kafka.producer;
+import com.netflix.servo.util.VisibleForTesting;
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.slf4j.Logger;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
+import org.slf4j.LoggerFactory;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaAction.COUNTERS_AND_TIMER;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaAction.OBJECT_POOL;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaAction.POSTS_IN_FLIGHT_COUNTER_INDEX;
-
-@Component
-public class ProduceIntoExternalKafkaCallback implements Callback {
+public class KafkaCallback implements Callback {
static final String DEBUG_MSG = "Successfully posted JSON to Kafka: topic [%s] partition [%d] offset [%d]";
static final String ERROR_MSG_TEMPLATE = "Callback exception posting JSON to Kafka; received message [%s]";
static final String POOL_ERROR_MSG_TEMPLATE = "Exception returning callback to pool; received message [%s]";
- private final Logger logger;
+ @VisibleForTesting
+ static Logger logger = LoggerFactory.getLogger(KafkaCallback.class);
- @Autowired
- public ProduceIntoExternalKafkaCallback(Logger produceIntoExternalKafkaCallbackLogger) {
- this.logger = produceIntoExternalKafkaCallbackLogger;
- }
@Override
public void onCompletion(RecordMetadata metadata, Exception exception) {
@@ -59,8 +51,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) {
private void returnObjectToPoolButLogExceptionIfReturnFails() {
try {
- COUNTERS_AND_TIMER.get().incrementCounter(POSTS_IN_FLIGHT_COUNTER_INDEX, -1);
- OBJECT_POOL.returnObject(this);
+ KafkaToKafkaPipeline.OBJECT_POOL.returnObject(this);
} catch (Exception exception) {
logError(exception, POOL_ERROR_MSG_TEMPLATE);
}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetrics.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetrics.java
new file mode 100644
index 0000000..ed723f7
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetrics.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+
+/*
+ This class handles request, success and failure metrics for kafka producer
+ */
+public class KafkaProducerMetrics {
+
+ private String name;
+ private Counter successCounter;
+ private Counter failureCounter;
+ private Counter requestCounter;
+ private Timer timer;
+
+ public KafkaProducerMetrics(String name, MetricRegistry metricRegistry) {
+ this.name = name;
+ this.requestCounter = metricRegistry.counter(name + "_requests_counter");
+ this.successCounter = metricRegistry.counter(name + "_success_counter");
+ this.failureCounter = metricRegistry.counter(name + "_failure_counter");
+ this.timer = metricRegistry.timer(name + "_timer");
+ }
+
+ public void incSuccessCounter() {
+ this.successCounter.inc();
+ }
+
+ public void incFailureCounter() {
+ this.failureCounter.inc();
+ }
+
+ public void incRequestCounter() {
+ this.requestCounter.inc();
+ }
+
+ public Timer getTimer() {
+ return timer;
+ }
+}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerWrapper.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerWrapper.java
new file mode 100644
index 0000000..662271e
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerWrapper.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+
+/*
+This class wraps kafka Producer with default topic defined in base.conf
+ */
+public class KafkaProducerWrapper {
+ private String defaultTopic;
+ private String name;
+ private KafkaProducer kafkaProducer;
+ private KafkaProducerMetrics kafkaProducerMetrics;
+
+ public KafkaProducerWrapper(String defaultTopic, String name, KafkaProducer kafkaProducer, KafkaProducerMetrics kafkaProducerMetrics) {
+ this.defaultTopic = defaultTopic;
+ this.name = name;
+ this.kafkaProducer = kafkaProducer;
+ this.kafkaProducerMetrics = kafkaProducerMetrics;
+ }
+
+ public String getDefaultTopic() {
+ return defaultTopic;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public KafkaProducer getKafkaProducer() {
+ return kafkaProducer;
+ }
+
+ public KafkaProducerMetrics getKafkaProducerMetrics() {
+ return kafkaProducerMetrics;
+ }
+}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipeline.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipeline.java
new file mode 100644
index 0000000..39b3bd5
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipeline.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2018 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.Timer;
+import com.expedia.open.tracing.Span;
+import com.expedia.www.haystack.pipes.commons.kafka.TagFlattener;
+import com.expedia.www.haystack.pipes.key.extractor.Record;
+import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
+import com.netflix.servo.util.VisibleForTesting;
+import org.apache.commons.pool2.ObjectPool;
+import org.apache.commons.pool2.impl.GenericObjectPool;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.streams.kstream.ForeachAction;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.util.CollectionUtils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+public class KafkaToKafkaPipeline implements ForeachAction {
+ @VisibleForTesting
+ static final String ERROR_MSG =
+ "Exception posting JSON [%s] to Kafka; received message [%s]";
+ static ObjectPool OBJECT_POOL = new GenericObjectPool<>(
+ new CallbackFactory());
+ @VisibleForTesting
+ static Logger logger = LoggerFactory.getLogger(KafkaToKafkaPipeline.class);
+ @VisibleForTesting
+ static Factory factory = new KafkaToKafkaPipeline.Factory();
+ static String kafkaProducerMsg = "Kafka message sent on topic: {}";
+ private final TagFlattener tagFlattener = new TagFlattener();
+ private final List spanKeyExtractors;
+ private final List kafkaProducerWrappers;
+
+
+ public KafkaToKafkaPipeline(List spanKeyExtractors,
+ List kafkaProducerWrappers) {
+ this.spanKeyExtractors = spanKeyExtractors;
+ this.kafkaProducerWrappers = kafkaProducerWrappers;
+ }
+
+ @Override
+ public void apply(String key, Span value) {
+ spanKeyExtractors.forEach(spanKeyExtractor -> {
+ List records = spanKeyExtractor.getRecords(value);
+ if (CollectionUtils.isEmpty(records))
+ logger.debug("Extractor skipped the span: {}", value);
+ records.forEach(record -> {
+ String message = record.getMessage();
+ final String kafkaKey = record.getKey();
+ String msgWithFlattenedTags = tagFlattener.flattenTags(message);
+ kafkaProducerWrappers.forEach(kafkaProducerWrapper -> {
+ kafkaProducerWrapper.getKafkaProducerMetrics().incRequestCounter();
+ List kafkaTopics = new ArrayList<>();
+ kafkaTopics.add(kafkaProducerWrapper.getDefaultTopic());
+ if (record.getProducerTopicsMappings() == null || !record.getProducerTopicsMappings().containsKey(kafkaProducerWrapper.getName())) {
+ logger.error("Extractor skipped the span: {}, as no topics found for producer: {}", value, kafkaProducerWrapper.getName());
+ return;
+ }
+ logger.debug("Kafka Producer sending message: {},with key: {} ", msgWithFlattenedTags, kafkaKey);
+ kafkaTopics.addAll(record.getProducerTopicsMappings().get(kafkaProducerWrapper.getName()));
+ produceToKafkaTopics(kafkaProducerWrapper.getKafkaProducer(), kafkaTopics, kafkaKey, msgWithFlattenedTags,
+ kafkaProducerWrapper.getKafkaProducerMetrics());
+ });
+ });
+
+ });
+ }
+
+ public void produceToKafkaTopics(KafkaProducer kafkaProducer, List kafkaTopics, String kafkaKey,
+ String jsonWithFlattenedTags, KafkaProducerMetrics kafkaProducerMetrics) {
+ Timer.Context time = kafkaProducerMetrics.getTimer().time();
+ kafkaTopics.forEach(topic -> {
+ final ProducerRecord producerRecord =
+ factory.createProducerRecord(topic, kafkaKey, jsonWithFlattenedTags);
+
+ final KafkaCallback callback; // callback must returnObject()
+ try {
+ callback = OBJECT_POOL.borrowObject();
+ kafkaProducerMetrics.incSuccessCounter();
+ // TODO Put the Span value into the callback so that it can write it to Kafka for retry
+ kafkaProducer.send(producerRecord, callback);
+ logger.debug(kafkaProducerMsg, topic);
+ } catch (Exception exception) {
+ kafkaProducerMetrics.incFailureCounter();
+ // Must format below because log4j2 underneath slf4j doesn't handle .error(varargs) properly
+ final String message = String.format(ERROR_MSG, jsonWithFlattenedTags, exception.getMessage());
+ logger.error(message, exception);
+ }
+ });
+ time.stop();
+ }
+
+
+ static class Factory {
+ ProducerRecord createProducerRecord(String topic, String key, String value) {
+ return new ProducerRecord<>(topic, key, value);
+ }
+
+ KafkaProducer createKafkaProducer(Map configurationMap) {
+ return new KafkaProducer<>(configurationMap);
+ }
+ }
+
+}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfiguration.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfiguration.java
new file mode 100644
index 0000000..568bbe0
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfiguration.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.expedia.www.haystack.commons.config.ConfigurationLoader;
+import com.expedia.www.haystack.pipes.commons.kafka.config.KafkaConsumerConfig;
+import com.expedia.www.haystack.pipes.kafka.producer.config.KafkaProducerConfig;
+import com.netflix.servo.util.VisibleForTesting;
+import com.typesafe.config.Config;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ProjectConfiguration {
+
+ @VisibleForTesting
+ static ProjectConfiguration projectConfiguration = null;
+ private static KafkaConsumerConfig kafkaConsumerConfig = null;
+ private static List kafkaProducerConfigs = null;
+ private static Map spanKeyExtractorConfigs = null;
+ private final Config haystackConfig;
+
+ private ProjectConfiguration() {
+ Config config = ConfigurationLoader.loadConfigFileWithEnvOverrides("config/base.conf", "HAYSTACK_PROP_");
+ haystackConfig = config.getConfig("haystack");
+ }
+
+ public static ProjectConfiguration getInstance() {
+ if (null == projectConfiguration) {
+ projectConfiguration = new ProjectConfiguration();
+ }
+ return projectConfiguration;
+ }
+
+ public KafkaConsumerConfig getKafkaConsumerConfig() {
+ if (null == kafkaConsumerConfig) {
+ Config kafkaConfig = haystackConfig.getConfig("kafka");
+ kafkaConsumerConfig = new KafkaConsumerConfig(kafkaConfig.getString("brokers"),
+ kafkaConfig.getInt("port"), kafkaConfig.getString("fromtopic"),
+ kafkaConfig.getString("totopic"), kafkaConfig.getInt("threadcount"),
+ kafkaConfig.getInt("sessiontimeout"), kafkaConfig.getInt("maxwakeups"),
+ kafkaConfig.getInt("wakeuptimeoutms"), kafkaConfig.getLong("polltimeoutms"),
+ kafkaConfig.getLong("commitms"));
+ }
+ return kafkaConsumerConfig;
+ }
+
+ public List getKafkaProducerConfigs() {
+ if (null == kafkaProducerConfigs) {
+ kafkaProducerConfigs = new ArrayList<>();
+ List kafkaConfigs = (List) haystackConfig.getConfigList("kafka.sinks");
+ kafkaConfigs.forEach(kafkaConfig -> {
+ kafkaProducerConfigs.add(new KafkaProducerConfig(kafkaConfig.getString("name"), kafkaConfig.getString("brokers"),
+ kafkaConfig.getInt("port"), kafkaConfig.getString("acks"), kafkaConfig.getInt("batchsize"),
+ kafkaConfig.getInt("lingerms"), kafkaConfig.getInt("buffermemory"), kafkaConfig.getString("defaultTopic")));
+ });
+ }
+ return kafkaProducerConfigs;
+ }
+
+ public Map getSpanExtractorConfigs() {
+ if (null == spanKeyExtractorConfigs) {
+ spanKeyExtractorConfigs = new HashMap<>();
+ List extractorConfigs = (List) haystackConfig.getConfigList("extractors");
+ extractorConfigs.forEach(extractorConfig -> {
+ String name = extractorConfig.getString("name");
+ Config config = extractorConfig.getConfig("config");
+ spanKeyExtractorConfigs.put(name, config);
+ });
+ }
+ return spanKeyExtractorConfigs;
+ }
+
+}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducer.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducer.java
similarity index 55%
rename from kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducer.java
rename to kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducer.java
index 10e8652..aed19da 100644
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducer.java
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducer.java
@@ -14,24 +14,18 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.kafkaProducer;
+package com.expedia.www.haystack.pipes.kafka.producer;
-import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamBuilderBase;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
+import com.expedia.www.haystack.pipes.commons.kafka.config.KafkaConsumerConfig;
import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-import static com.expedia.www.haystack.pipes.kafkaProducer.Constants.APPLICATION;
-
-@Component
public class ProtobufToKafkaProducer extends KafkaStreamBuilderBase {
- @Autowired
- ProtobufToKafkaProducer(KafkaStreamStarter kafkaStreamStarter,
- SerdeFactory serdeFactory,
- ProduceIntoExternalKafkaAction produceIntoExternalKafkaAction,
- KafkaConfigurationProvider kafkaConfigurationProvider) {
- super(kafkaStreamStarter, serdeFactory, APPLICATION, kafkaConfigurationProvider, produceIntoExternalKafkaAction);
+ public ProtobufToKafkaProducer(KafkaStreamStarter kafkaStreamStarter,
+ SerdeFactory serdeFactory,
+ KafkaToKafkaPipeline kafkaToKafkaPipeline,
+ KafkaConsumerConfig kafkaConsumerConfig) {
+ super(kafkaStreamStarter, serdeFactory, Constants.APPLICATION, kafkaConsumerConfig, kafkaToKafkaPipeline);
}
}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Service.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Service.java
new file mode 100644
index 0000000..3b1ce0b
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/Service.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.JmxReporter;
+import com.codahale.metrics.MetricRegistry;
+import com.expedia.www.haystack.pipes.commons.health.HealthController;
+import com.expedia.www.haystack.pipes.commons.health.UpdateHealthStatusFile;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
+import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
+import com.expedia.www.haystack.pipes.kafka.producer.config.KafkaProducerConfig;
+import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
+import com.expedia.www.haystack.pipes.key.extractor.loader.SpanKeyExtractorLoader;
+import com.netflix.servo.util.VisibleForTesting;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+
+public class Service {
+
+ private static final SerdeFactory serdeFactory = new SerdeFactory();
+ private static final MetricRegistry metricRegistry = new MetricRegistry();
+ private static final ProjectConfiguration projectConfiguration = ProjectConfiguration.getInstance();
+ @VisibleForTesting
+ static Logger logger = LoggerFactory.getLogger(Service.class);
+ @VisibleForTesting
+ static HealthController healthController = new HealthController();
+ @VisibleForTesting
+ static Service service = null;
+ @VisibleForTesting
+ private static List kafkaProducerWrappers = null;
+
+ private Service() {
+ }
+
+ public static void main(String[] args) {
+ logger.info("Initializing Kafka Consumers");
+ service = Service.getInstance();
+ service.inPlaceHealthCheck();
+ JmxReporter jmxReporter = service.getJmxReporter();
+ jmxReporter.start();
+ final ProtobufToKafkaProducer protobufToKafkaProducer = service.getProtobufToKafkaProducer(service.getKafkaStreamStarter());
+ protobufToKafkaProducer.main();
+ }
+
+ public static Service getInstance() {
+ if (null == service) {
+ service = new Service();
+ }
+ return service;
+ }
+
+ public static List getKafkaProducerWrappers(ProjectConfiguration projectConfiguration) {
+
+ if (null == kafkaProducerWrappers) {
+ List kafkaProducerConfigs = projectConfiguration.getKafkaProducerConfigs();
+ KafkaToKafkaPipeline.Factory factory = new KafkaToKafkaPipeline.Factory();
+
+ kafkaProducerWrappers = kafkaProducerConfigs.stream()
+ .map(kafkaProducerConfig -> {
+ KafkaProducer kafkaProducer = factory.createKafkaProducer(kafkaProducerConfig.getConfigurationMap());
+ KafkaProducerMetrics kafkaProducerMetrics = new KafkaProducerMetrics(kafkaProducerConfig.getName(), metricRegistry);
+ return new KafkaProducerWrapper(kafkaProducerConfig.getDefaultTopic(), kafkaProducerConfig.getName(), kafkaProducer, kafkaProducerMetrics);
+ }).collect(Collectors.toList());
+ }
+
+ return kafkaProducerWrappers;
+ }
+
+ public ProtobufToKafkaProducer getProtobufToKafkaProducer(KafkaStreamStarter kafkaStreamStarter) {
+ return new ProtobufToKafkaProducer(kafkaStreamStarter,
+ serdeFactory, getKafkaToKafkaPipeline(),
+ projectConfiguration.getKafkaConsumerConfig());
+ }
+
+ public KafkaStreamStarter getKafkaStreamStarter() {
+ return new KafkaStreamStarter(ProtobufToKafkaProducer.class,
+ Constants.APPLICATION, projectConfiguration.getKafkaConsumerConfig(),
+ healthController);
+ }
+
+ KafkaToKafkaPipeline getKafkaToKafkaPipeline() {
+ List spanKeyExtractors = SpanKeyExtractorLoader.getInstance().getSpanKeyExtractor(projectConfiguration.getSpanExtractorConfigs());
+ return new KafkaToKafkaPipeline(spanKeyExtractors, getKafkaProducerWrappers(projectConfiguration));
+ }
+
+ public void inPlaceHealthCheck() {
+ healthController.addListener(new UpdateHealthStatusFile("/app/isHealthy"));
+ }
+
+ public JmxReporter getJmxReporter() {
+ return JmxReporter.forRegistry(metricRegistry).build();
+ }
+
+}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/config/KafkaProducerConfig.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/config/KafkaProducerConfig.java
new file mode 100644
index 0000000..ac7fe2e
--- /dev/null
+++ b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/config/KafkaProducerConfig.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer.config;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.common.serialization.StringSerializer;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class KafkaProducerConfig {
+
+ private String name;
+
+ private String brokers;
+
+ private int port;
+
+ private String acks;
+
+ private int batchSize;
+
+ private int lingerMs;
+
+ private int bufferMemory;
+
+ private String defaultTopic;
+
+ public KafkaProducerConfig(String name, String brokers, int port,
+ String acks, int batchSize, int lingerMs,
+ int bufferMemory, String defaultTopic) {
+ this.name = name;
+ this.brokers = brokers;
+ this.port = port;
+ this.acks = acks;
+ this.batchSize = batchSize;
+ this.lingerMs = lingerMs;
+ this.bufferMemory = bufferMemory;
+ this.defaultTopic = defaultTopic;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getBrokers() {
+ return this.brokers;
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public String getAcks() {
+ return this.acks;
+ }
+
+ public int getBatchSize() {
+ return this.batchSize;
+ }
+
+ public int getLingerMs() {
+ return this.lingerMs;
+ }
+
+ public int getBufferMemory() {
+ return this.bufferMemory;
+ }
+
+ public String getDefaultTopic() {
+ return defaultTopic;
+ }
+
+ public Map getConfigurationMap() {
+ final Map map = new HashMap<>();
+ map.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, this.getBrokers());
+ map.put(ProducerConfig.ACKS_CONFIG, getAcks());
+ map.put(ProducerConfig.RETRIES_CONFIG, 3);
+ map.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, 1000);
+ map.put(ProducerConfig.BATCH_SIZE_CONFIG, getBatchSize());
+ map.put(ProducerConfig.LINGER_MS_CONFIG, getLingerMs());
+ map.put(ProducerConfig.BUFFER_MEMORY_CONFIG, this.getBufferMemory());
+ map.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+ map.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+ return map;
+ }
+}
\ No newline at end of file
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/CallbackFactory.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/CallbackFactory.java
deleted file mode 100644
index dc6e720..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/CallbackFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import org.apache.commons.pool2.BasePooledObjectFactory;
-import org.apache.commons.pool2.PooledObject;
-import org.apache.commons.pool2.impl.DefaultPooledObject;
-import org.slf4j.Logger;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-@Component
-public class CallbackFactory extends BasePooledObjectFactory {
- private final Logger produceIntoExternalKafkaCallbackLogger;
-
- @Autowired
- CallbackFactory(Logger produceIntoExternalKafkaCallbackLogger) {
- this.produceIntoExternalKafkaCallbackLogger = produceIntoExternalKafkaCallbackLogger;
- }
-
- @Override
- public ProduceIntoExternalKafkaCallback create() {
- return new ProduceIntoExternalKafkaCallback(produceIntoExternalKafkaCallbackLogger);
- }
-
- @Override
- public PooledObject wrap(
- ProduceIntoExternalKafkaCallback produceIntoExternalKafkaCallback) {
- return new DefaultPooledObject<>(produceIntoExternalKafkaCallback);
- }
-}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfig.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfig.java
deleted file mode 100644
index 409bf87..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfig.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-/**
- * Configurations for the KafkaProducer that sends data to an Kafka outside of the Haystack system.
- * For details on this configurations, see Kafka documentation, e.g.
- * http://kafka.apache.org/documentation.html#producerconfigs,
- * https://kafka.apache.org/0102/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html, etc.
- * All method names must be lower case and must not include underscores, because the cf4j framework that manages
- * configuration changes configurations specified by environment variables from upper to lower case and replaces
- * underscores by periods.
- */
-public interface ExternalKafkaConfig {
- String brokers();
-
- int port();
-
- String totopic();
-
- String acks(); // "-1": all replicas; "0": don't wait; "1": leader writes to its local log; "all": same as "-1"
-
- int batchsize();
-
- int lingerms();
-
- int buffermemory();
-}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProvider.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProvider.java
deleted file mode 100644
index 4905f2e..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProvider.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import com.expedia.www.haystack.commons.config.Configuration;
-import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.clients.producer.ProducerConfig;
-import org.apache.kafka.common.serialization.StringSerializer;
-import org.cfg4j.provider.ConfigurationProvider;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import static com.expedia.www.haystack.pipes.commons.Configuration.HAYSTACK_EXTERNAL_KAFKA_CONFIG_PREFIX;
-
-public class ExternalKafkaConfigurationProvider implements ExternalKafkaConfig {
- private ExternalKafkaConfig externalKafkaConfig;
-
- ExternalKafkaConfigurationProvider() {
- reload();
- }
-
- @Override
- public String brokers() {
- return externalKafkaConfig.brokers();
- }
-
- @Override
- public int port() {
- return externalKafkaConfig.port();
- }
-
- @Override
- public String totopic() {
- return externalKafkaConfig.totopic();
- }
-
- @Override
- public String acks() {
- return externalKafkaConfig.acks();
- }
-
- @Override
- public int batchsize() {
- return externalKafkaConfig.batchsize();
- }
-
- @Override
- public int lingerms() {
- return externalKafkaConfig.lingerms();
- }
-
- @Override
- public int buffermemory() {
- return externalKafkaConfig.buffermemory();
- }
-
- void reload() {
- final Configuration configuration = new Configuration();
- final ConfigurationProvider configurationProvider = configuration.createMergeConfigurationProvider();
- externalKafkaConfig = configurationProvider.bind(HAYSTACK_EXTERNAL_KAFKA_CONFIG_PREFIX, ExternalKafkaConfig.class);
- }
-
- Map getConfigurationMap() {
- final Map map = new HashMap<>();
- map.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokers());
- map.put(ProducerConfig.ACKS_CONFIG, acks());
- map.put(ProducerConfig.RETRIES_CONFIG, 3);
- map.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, 1000);
- map.put(ProducerConfig.BATCH_SIZE_CONFIG, batchsize());
- map.put(ProducerConfig.LINGER_MS_CONFIG, lingerms());
- map.put(ProducerConfig.BUFFER_MEMORY_CONFIG, buffermemory());
- map.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
- map.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
- return map;
- }
-
-
-}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveController.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveController.java
deleted file mode 100644
index 9673d4d..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveController.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import com.netflix.servo.util.VisibleForTesting;
-import org.slf4j.Logger;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.boot.SpringApplication;
-import org.springframework.boot.autoconfigure.SpringBootApplication;
-import org.springframework.boot.web.support.SpringBootServletInitializer;
-import org.springframework.context.annotation.AnnotationConfigApplicationContext;
-import org.springframework.stereotype.Component;
-
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * A very simple Spring Boot application that is intended to support only a single REST endpoint (index.html)
- * to indicate that the JVM is running.
- */
-@SpringBootApplication
-@Component
-public class KafkaProducerIsActiveController extends SpringBootServletInitializer {
- // Singleton, initialized on first constructor call, so that future instances created by Spring during unit tests
- // will not overwrite the initial INSTANCE (with mocks) created by the unit tests.
- @VisibleForTesting
- static final AtomicReference INSTANCE = new AtomicReference<>();
- @VisibleForTesting static final String STARTUP_MSG = "Starting FirehoseIsActiveController";
-
- private final ProtobufToKafkaProducer protobufToKafkaProducer;
- private final Factory factory;
- private final Logger logger;
-
- @Autowired
- KafkaProducerIsActiveController(ProtobufToKafkaProducer protobufToKafkaProducer,
- Factory kafkaProducerIsActiveControllerFactory,
- Logger kafkaProducerIsActiveControllerLogger) {
- this.protobufToKafkaProducer = protobufToKafkaProducer;
- this.factory = kafkaProducerIsActiveControllerFactory;
- this.logger = kafkaProducerIsActiveControllerLogger;
- INSTANCE.compareAndSet(null, this);
- }
-
- public static void main(String[] args) {
- new AnnotationConfigApplicationContext(SpringConfig.class);
- INSTANCE.get().logger.info(STARTUP_MSG);
- INSTANCE.get().protobufToKafkaProducer.main();
- INSTANCE.get().factory.createSpringApplication().run(args);
- }
-
- static class Factory {
- SpringApplication createSpringApplication() {
- return new SpringApplication(KafkaProducerIsActiveController.class);
- }
- }
-}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaAction.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaAction.java
deleted file mode 100644
index 3c23880..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaAction.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import com.expedia.open.tracing.Span;
-import com.expedia.www.haystack.pipes.commons.TimersAndCounters;
-import com.expedia.www.haystack.pipes.commons.kafka.TagFlattener;
-import com.google.protobuf.util.JsonFormat;
-import com.netflix.servo.monitor.Stopwatch;
-import com.netflix.servo.util.VisibleForTesting;
-import org.apache.commons.pool2.ObjectPool;
-import org.apache.commons.pool2.impl.GenericObjectPool;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.streams.kstream.ForeachAction;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.stereotype.Component;
-
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicReference;
-
-@Component
-public class ProduceIntoExternalKafkaAction implements ForeachAction {
- private static final JsonFormat.Printer printer = JsonFormat.printer().omittingInsignificantWhitespace();
- @VisibleForTesting
- static final String TOPIC_MESSAGE =
- "Loading ProduceIntoExternalKafkaAction with brokers [%s] port [%d] topic [%s]";
- static AtomicReference COUNTERS_AND_TIMER = new AtomicReference<>(null);
- static ObjectPool OBJECT_POOL = new GenericObjectPool<>(
- new CallbackFactory(LoggerFactory.getLogger(ProduceIntoExternalKafkaCallback.class)));
-
- @VisibleForTesting static final String ERROR_MSG =
- "Exception posting JSON [%s] to Kafka; received message [%s]";
- @VisibleForTesting static final int POSTS_IN_FLIGHT_COUNTER_INDEX = 0;
- private final Factory factory;
- private final TimersAndCounters timersAndCounters;
- private final Logger logger;
- private final KafkaProducer kafkaProducer;
- private final String topic;
-
- private final TagFlattener tagFlattener = new TagFlattener();
-
- @Autowired
- public ProduceIntoExternalKafkaAction(Factory produceIntoExternalKafkaActionFactory,
- TimersAndCounters timersAndCounters,
- Logger produceIntoExternalKafkaActionLogger,
- ExternalKafkaConfigurationProvider externalKafkaConfigurationProvider) {
- this.factory = produceIntoExternalKafkaActionFactory;
- this.timersAndCounters = timersAndCounters;
- COUNTERS_AND_TIMER.compareAndSet(null, timersAndCounters);
- this.logger = produceIntoExternalKafkaActionLogger;
-
- final Map configurationMap = externalKafkaConfigurationProvider.getConfigurationMap();
- this.kafkaProducer = factory.createKafkaProducer(configurationMap);
- this.topic = externalKafkaConfigurationProvider.totopic();
- logger.info(String.format(TOPIC_MESSAGE, externalKafkaConfigurationProvider.brokers(),
- externalKafkaConfigurationProvider.port(), topic));
- }
-
- @Override
- public void apply(String key, Span value) {
- timersAndCounters.incrementRequestCounter();
- final Stopwatch stopwatch = timersAndCounters.startTimer();
- String jsonWithFlattenedTags = "";
- try {
- final String jsonWithOpenTracingTags = printer.print(value);
- jsonWithFlattenedTags = tagFlattener.flattenTags(jsonWithOpenTracingTags);
- final ProducerRecord producerRecord =
- factory.createProducerRecord(topic, key, jsonWithFlattenedTags);
-
- final ProduceIntoExternalKafkaCallback callback = OBJECT_POOL.borrowObject(); // callback must returnObject()
- timersAndCounters.incrementCounter(POSTS_IN_FLIGHT_COUNTER_INDEX);
- // TODO Put the Span value into the callback so that it can write it to Kafka for retry
-
- kafkaProducer.send(producerRecord, callback);
- } catch (Exception exception) {
- // Must format below because log4j2 underneath slf4j doesn't handle .error(varargs) properly
- final String message = String.format(ERROR_MSG, jsonWithFlattenedTags, exception.getMessage());
- logger.error(message, exception);
- } finally {
- stopwatch.stop();
- }
- }
-
- static class Factory {
- ProducerRecord createProducerRecord(String topic, String key, String value) {
- return new ProducerRecord<>(topic, key, value);
- }
-
- KafkaProducer createKafkaProducer(Map configurationMap) {
- return new KafkaProducer<>(configurationMap);
- }
- }
-
-}
diff --git a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/SpringConfig.java b/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/SpringConfig.java
deleted file mode 100644
index 49cbcff..0000000
--- a/kafka-producer/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/SpringConfig.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import com.expedia.www.haystack.metrics.MetricObjects;
-import com.expedia.www.haystack.pipes.commons.Timers;
-import com.expedia.www.haystack.pipes.commons.TimersAndCounters;
-import com.expedia.www.haystack.pipes.commons.health.HealthController;
-import com.expedia.www.haystack.pipes.commons.health.UpdateHealthStatusFile;
-import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
-import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
-import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
-import com.netflix.servo.monitor.Counter;
-import com.netflix.servo.monitor.Timer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.ComponentScan;
-import org.springframework.context.annotation.Configuration;
-
-import java.time.Clock;
-import java.util.concurrent.TimeUnit;
-
-import static com.expedia.www.haystack.pipes.commons.CommonConstants.SPAN_ARRIVAL_TIMER_NAME;
-import static com.expedia.www.haystack.pipes.commons.CommonConstants.SUBSYSTEM;
-import static com.expedia.www.haystack.pipes.kafkaProducer.Constants.APPLICATION;
-
-@Configuration
-@ComponentScan(basePackageClasses = SpringConfig.class)
-public class SpringConfig {
-
- private final MetricObjects metricObjects;
-
- /**
- * @param metricObjects provided by a static inner class that is loaded first
- * @see MetricObjectsSpringConfig
- */
- @Autowired
- SpringConfig(MetricObjects metricObjects) {
- this.metricObjects = metricObjects;
- }
-
- // Beans with unit tests ///////////////////////////////////////////////////////////////////////////////////////////
- @Bean
- Counter produceIntoExternalKafkaActionRequestCounter() {
- return metricObjects.createAndRegisterResettingCounter(SUBSYSTEM, APPLICATION,
- ProduceIntoExternalKafkaAction.class.getSimpleName(), "REQUEST");
- }
-
- @Bean
- Counter postsInFlightCounter() {
- return metricObjects.createAndRegisterResettingCounter(SUBSYSTEM, APPLICATION,
- ProduceIntoExternalKafkaAction.class.getSimpleName(), "POSTS_IN_FLIGHT");
- }
-
- @Bean
- Logger produceIntoExternalKafkaCallbackLogger() {
- return LoggerFactory.getLogger(ProduceIntoExternalKafkaCallback.class);
- }
-
- @Bean
- Logger produceIntoExternalKafkaActionLogger() {
- return LoggerFactory.getLogger(ProduceIntoExternalKafkaAction.class);
- }
-
- @Bean
- Logger kafkaProducerIsActiveControllerLogger() {
- return LoggerFactory.getLogger(KafkaProducerIsActiveController.class);
- }
-
- @Bean
- @Autowired
- KafkaStreamStarter kafkaStreamStarter(final HealthController healthController) {
- return new KafkaStreamStarter(ProtobufToKafkaProducer.class, APPLICATION, healthController);
- }
-
- @Bean
- HealthController healthController() {
- final HealthController healthController = new HealthController();
- healthController.addListener(new UpdateHealthStatusFile("/app/isHealthy")); // TODO should come from config
- return healthController;
- }
-
- @Bean
- Timer kafkaProducerPost() {
- return metricObjects.createAndRegisterBasicTimer(SUBSYSTEM, APPLICATION,
- ProduceIntoExternalKafkaAction.class.getSimpleName(), "KAFKA_PRODUCER_POST", TimeUnit.MICROSECONDS);
- }
-
- @Bean
- Timer spanArrivalTimer() {
- return metricObjects.createAndRegisterBasicTimer(SUBSYSTEM, APPLICATION,
- ProduceIntoExternalKafkaAction.class.getSimpleName(), SPAN_ARRIVAL_TIMER_NAME, TimeUnit.MILLISECONDS);
- }
-
- // Beans without unit tests ////////////////////////////////////////////////////////////////////////////////////////
- @Bean
- SerdeFactory serdeFactory() {
- return new SerdeFactory();
- }
-
- @Bean
- @Autowired
- CallbackFactory callbackFactory(Logger produceIntoExternalKafkaCallbackLogger) {
- return new CallbackFactory(produceIntoExternalKafkaCallbackLogger);
- }
-
- @Bean
- KafkaConfigurationProvider kafkaConfigurationProvider() {
- return new KafkaConfigurationProvider();
- }
-
- @Bean
- ExternalKafkaConfigurationProvider externalKafkaConfigurationProvider() {
- return new ExternalKafkaConfigurationProvider();
- }
-
- @Bean
- KafkaProducerIsActiveController.Factory kafkaProducerIsActiveControllerFactory() {
- return new KafkaProducerIsActiveController.Factory();
- }
-
- @Bean
- ProduceIntoExternalKafkaAction.Factory produceIntoExternalKafkaActionFactory() {
- return new ProduceIntoExternalKafkaAction.Factory();
- }
-
- @Bean
- Clock clock() {
- return Clock.systemUTC();
- }
-
- @Bean
- @Autowired
- Timers timers(Timer kafkaProducerPost,
- Timer spanArrivalTimer) {
- return new Timers(kafkaProducerPost, spanArrivalTimer);
- }
-
- @Bean
- @Autowired
- TimersAndCounters countersAndTimer(Clock clock,
- Counter produceIntoExternalKafkaActionRequestCounter,
- Counter postsInFlightCounter,
- Timers timers) {
- return new TimersAndCounters(clock, timers,
- produceIntoExternalKafkaActionRequestCounter, postsInFlightCounter);
- }
-
- @Bean
- @Autowired
- ProduceIntoExternalKafkaAction produceIntoExternalKafkaAction(
- ProduceIntoExternalKafkaAction.Factory produceIntoExternalKafkaActionFactoryFactory,
- TimersAndCounters timersAndCounters,
- Logger produceIntoExternalKafkaActionLogger,
- ExternalKafkaConfigurationProvider externalKafkaConfigurationProvider) {
- return new ProduceIntoExternalKafkaAction(
- produceIntoExternalKafkaActionFactoryFactory,
- timersAndCounters,
- produceIntoExternalKafkaActionLogger,
- externalKafkaConfigurationProvider);
- }
-
- @Bean
- @Autowired
- ProtobufToKafkaProducer protobufToKafkaProducer(KafkaStreamStarter kafkaStreamStarter,
- SerdeFactory serdeFactory,
- ProduceIntoExternalKafkaAction produceIntoExternalKafkaAction,
- KafkaConfigurationProvider kafkaConfigurationProvider) {
- return new ProtobufToKafkaProducer(
- kafkaStreamStarter, serdeFactory, produceIntoExternalKafkaAction, kafkaConfigurationProvider);
- }
-
- /*
- * Spring loads this static inner class before loading the SpringConfig outer class so that its bean is available to
- * the outer class constructor.
- *
- * @see SpringConfig
- */
- @Configuration
- static class MetricObjectsSpringConfig {
- @Bean
- public MetricObjects metricObjects() {
- return new MetricObjects();
- }
- }
-}
diff --git a/kafka-producer/src/main/resources/base.yaml b/kafka-producer/src/main/resources/base.yaml
deleted file mode 100644
index 233cbea..0000000
--- a/kafka-producer/src/main/resources/base.yaml
+++ /dev/null
@@ -1,30 +0,0 @@
-haystack:
- kafka:
- brokers: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
- port: 9092 # default Kafka port, rarely overridden, but can be overridden by env variable
- fromtopic: "proto-spans"
- totopic: "" # not used but must be specified as empty for shared YAML parsing to work
- threadcount: 1
- sessiontimeout: 15000
- maxwakeups: 10
- wakeuptimeoutms: 3000
- polltimeoutms: 250
- commitms: 3000
- pipe:
- streams:
- replicationfactor: 1
- graphite:
- prefix: "haystack"
- host: "haystack.local" # set in /etc/hosts per instructions in haystack-deployment package
- port: 2003 # default Graphite port, rarely overridden, but can be overridden by env variable
- pollintervalseconds: 300
- queuesize: 10
- sendasrate: false
- externalkafka:
- brokers: "haystack.local:9092" # will point to external Kafka in typical usage
- port: 9092
- totopic: "externalKafkaTopic"
- acks: "0"
- batchsize: 16384
- lingerms: 5
- buffermemory: 1048576 # 1 Megabyte
diff --git a/kafka-producer/src/main/resources/config/base.conf b/kafka-producer/src/main/resources/config/base.conf
index c92bc52..135f1b1 100644
--- a/kafka-producer/src/main/resources/config/base.conf
+++ b/kafka-producer/src/main/resources/config/base.conf
@@ -1,13 +1,42 @@
{
"haystack": {
- "directory": "extractors/"
+ "kafka": {
+ "brokers": "localhost",
+ "port": 9092,
+ "fromtopic": "proto-spans",
+ "totopic": "",
+ "threadcount": 1,
+ "sessiontimeout": 15000,
+ "maxwakeups": 10,
+ "wakeuptimeoutms": 3000,
+ "polltimeoutms": 250,
+ "commitms": 3000,
+ // Configuration for the Kafka Producer. Documentation for these configurations can be found in the
+ // http://kafka.apache.org/documentation.html#producerconfigs
+ "sinks": [{// list of available kafka producers
+ "name": "DefaultKafkaProducer" // name to identify for the kafka producer- extractor mapping
+ "brokers": "localhost:9092",
+ "port": 9093,
+ "defaultTopic": "externalKafkaTopic", // default topic for producer
+ "acks": "0",
+ "batchsize": 8192,
+ "lingerms": 4,
+ "buffermemory": 9000
+ }]
+ },
"extractors": [
{
- "name": "SampleExtractor",
+ "name": "SampleExtractor",// name of extractor class
"config": {
- "dummyKeyPair": "dummyValuePair"
- }
+ "producers": [{
+ "name": "DefaultKafkaProducer",//
+ "config": {
+ "topics": ["firstTopic", "secondTopic"]
+ } // configurations required by extractor to manipulate key, topic or message
+ }] // list of producers: here it represents kafka producer for the extractor
+ },// configuration needed by extractor class
}
]
}
}
+
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallbackTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallbackTest.java
similarity index 56%
rename from kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallbackTest.java
rename to kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallbackTest.java
index 22a9adf..d20c7cc 100644
--- a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProduceIntoExternalKafkaCallbackTest.java
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaCallbackTest.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2018 Expedia, Inc.
+ * Copyright 2020 Expedia, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -14,12 +14,8 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.kafkaProducer;
+package com.expedia.www.haystack.pipes.kafka.producer;
-import com.expedia.www.haystack.pipes.commons.Timers;
-import com.expedia.www.haystack.pipes.commons.TimersAndCounters;
-import com.netflix.servo.monitor.Counter;
-import com.netflix.servo.monitor.Timer;
import org.apache.commons.pool2.ObjectPool;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
@@ -31,24 +27,14 @@
import org.mockito.runners.MockitoJUnitRunner;
import org.slf4j.Logger;
-import java.time.Clock;
-
import static com.expedia.www.haystack.pipes.commons.test.TestConstantsAndCommonCode.RANDOM;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaAction.COUNTERS_AND_TIMER;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaAction.OBJECT_POOL;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaCallback.DEBUG_MSG;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaCallback.ERROR_MSG_TEMPLATE;
-import static com.expedia.www.haystack.pipes.kafkaProducer.ProduceIntoExternalKafkaCallback.POOL_ERROR_MSG_TEMPLATE;
import static org.junit.Assert.assertSame;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.doThrow;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
@RunWith(MockitoJUnitRunner.class)
-public class ProduceIntoExternalKafkaCallbackTest {
+public class KafkaCallbackTest {
private static final String TOPIC = RANDOM.nextLong() + "TOPIC";
private static final int PARTITION = RANDOM.nextInt();
private static final TopicPartition TOPIC_PARTITION = new TopicPartition(TOPIC, PARTITION);
@@ -67,74 +53,53 @@ public class ProduceIntoExternalKafkaCallbackTest {
private Exception mockException;
@Mock
- private ObjectPool mockObjectPool;
- private ObjectPool realObjectPool;
-
- @Mock
- private Counter mockRequestCounter;
-
- @Mock
- private Counter mockPostsInFlightCounter;
-
- @Mock
- private Timer mockTimer;
-
- @Mock
- private Clock mockClock;
-
- @Mock
- private Timer mockSpanArrivalTimer;
+ private ObjectPool mockObjectPool;
+ private ObjectPool realObjectPool;
- private TimersAndCounters timersAndCounters;
private RecordMetadata recordMetadata;
- private ProduceIntoExternalKafkaCallback produceIntoExternalKafkaCallback;
+ private KafkaCallback kafkaCallback;
@Before
public void setUp() {
- Timers timers = new Timers(mockTimer, mockSpanArrivalTimer);
- timersAndCounters = new TimersAndCounters(mockClock, timers, mockRequestCounter, mockPostsInFlightCounter);
injectMockAndSaveRealObjects();
//noinspection deprecation
recordMetadata = new RecordMetadata(TOPIC_PARTITION, BASE_OFFSET, RELATIVE_OFFSET, TIMESTAMP, CHECKSUM,
SERIALIZED_KEY_SIZE, SERIALIZED_VALUE_SIZE);
- produceIntoExternalKafkaCallback = new ProduceIntoExternalKafkaCallback(mockLogger);
+ kafkaCallback = new KafkaCallback();
+ KafkaCallback.logger = mockLogger;
}
private void injectMockAndSaveRealObjects() {
saveRealAndInjectMockObjectPool();
- COUNTERS_AND_TIMER.set(timersAndCounters);
}
private void saveRealAndInjectMockObjectPool() {
- realObjectPool = OBJECT_POOL;
- OBJECT_POOL = mockObjectPool;
+ realObjectPool = KafkaToKafkaPipeline.OBJECT_POOL;
+ KafkaToKafkaPipeline.OBJECT_POOL = mockObjectPool;
}
@After
public void tearDown() {
restoreRealObjects();
- COUNTERS_AND_TIMER.set(null);
- verifyNoMoreInteractions(mockLogger, mockException, mockObjectPool, mockRequestCounter,
- mockPostsInFlightCounter, mockTimer, mockClock, mockSpanArrivalTimer);
}
private void restoreRealObjects() {
- OBJECT_POOL = realObjectPool;
+ KafkaToKafkaPipeline.OBJECT_POOL = realObjectPool;
}
@Test
public void testOnCompletionBothNull() throws Exception {
- produceIntoExternalKafkaCallback.onCompletion(null, null);
+ kafkaCallback.onCompletion(null, null);
commonVerifiesForOnCompletion();
}
@Test
public void testOnCompletionBothNullReturnToObjectPoolSuccess() throws Exception {
final Exception testException = new Exception("Exception Message");
- doThrow(testException).when(mockObjectPool).returnObject(any(ProduceIntoExternalKafkaCallback.class));
+ doThrow(testException).when(mockObjectPool).returnObject(any(KafkaCallback.class));
- produceIntoExternalKafkaCallback.onCompletion(null, null);
- verify(mockLogger).error(String.format(POOL_ERROR_MSG_TEMPLATE, testException.getMessage()), testException);
+ kafkaCallback.onCompletion(null, null);
+ verify(mockLogger).error(String.format(KafkaCallback.POOL_ERROR_MSG_TEMPLATE, testException.getMessage()), testException);
commonVerifiesForOnCompletion();
}
@@ -145,10 +110,10 @@ public void testOnCompletionRuntimeExceptionReturnToObjectPoolSuccess() throws E
doThrow(runtimeException).when(mockLogger).error(anyString(), any(Throwable.class));
try {
- produceIntoExternalKafkaCallback.onCompletion(null, testException);
- } catch(Throwable e) {
+ kafkaCallback.onCompletion(null, testException);
+ } catch (Throwable e) {
assertSame(runtimeException, e);
- verify(mockLogger).error(String.format(ERROR_MSG_TEMPLATE, testException.getMessage()), testException);
+ verify(mockLogger).error(String.format(KafkaCallback.ERROR_MSG_TEMPLATE, testException.getMessage()), testException);
commonVerifiesForOnCompletion();
}
}
@@ -157,7 +122,7 @@ public void testOnCompletionRuntimeExceptionReturnToObjectPoolSuccess() throws E
public void testOnCompletionNonNullMetadataDebugDisabled() throws Exception {
when(mockLogger.isDebugEnabled()).thenReturn(false);
- produceIntoExternalKafkaCallback.onCompletion(recordMetadata, null);
+ kafkaCallback.onCompletion(recordMetadata, null);
verify(mockLogger).isDebugEnabled();
commonVerifiesForOnCompletion();
@@ -167,10 +132,10 @@ public void testOnCompletionNonNullMetadataDebugDisabled() throws Exception {
public void testOnCompletionNonNullMetadataDebugEnabled() throws Exception {
when(mockLogger.isDebugEnabled()).thenReturn(true);
- produceIntoExternalKafkaCallback.onCompletion(recordMetadata, null);
+ kafkaCallback.onCompletion(recordMetadata, null);
verify(mockLogger).isDebugEnabled();
- verify(mockLogger).debug(String.format(DEBUG_MSG, TOPIC, PARTITION, BASE_OFFSET));
+ verify(mockLogger).debug(String.format(KafkaCallback.DEBUG_MSG, TOPIC, PARTITION, BASE_OFFSET));
commonVerifiesForOnCompletion();
}
@@ -178,16 +143,15 @@ public void testOnCompletionNonNullMetadataDebugEnabled() throws Exception {
public void testOneCompletionNonNullException() throws Exception {
when(mockException.getMessage()).thenReturn(MESSAGE);
- produceIntoExternalKafkaCallback.onCompletion(null, mockException);
+ kafkaCallback.onCompletion(null, mockException);
verify(mockException).getMessage();
- verify(mockLogger).error(String.format(ERROR_MSG_TEMPLATE, MESSAGE), mockException);
+ verify(mockLogger).error(String.format(KafkaCallback.ERROR_MSG_TEMPLATE, MESSAGE), mockException);
commonVerifiesForOnCompletion();
}
private void commonVerifiesForOnCompletion() throws Exception {
- verify(mockObjectPool).returnObject(produceIntoExternalKafkaCallback);
- verify(mockPostsInFlightCounter).increment(-1);
+ verify(mockObjectPool).returnObject(kafkaCallback);
}
}
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetricsTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetricsTest.java
new file mode 100644
index 0000000..a6b6c20
--- /dev/null
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerMetricsTest.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+
+@RunWith(MockitoJUnitRunner.class)
+public class KafkaProducerMetricsTest {
+
+ @Mock
+ private MetricRegistry mockMetricRegistry;
+ @Mock
+ private Counter mockSuccessCounter;
+ @Mock
+ private Counter mockFailureCounter;
+ @Mock
+ private Counter mockRequestCounter;
+ @Mock
+ private Timer mockTimer;
+
+ private KafkaProducerMetrics kafkaProducerMetrics;
+
+
+ @Before
+ public void setUp() {
+ when(mockMetricRegistry.counter("default_kafka_requests_counter")).thenReturn(mockRequestCounter);
+ when(mockMetricRegistry.counter("default_kafka_success_counter")).thenReturn(mockSuccessCounter);
+ when(mockMetricRegistry.counter("default_kafka_failure_counter")).thenReturn(mockFailureCounter);
+ when(mockMetricRegistry.timer("default_kafka_timer")).thenReturn(mockTimer);
+ kafkaProducerMetrics = new KafkaProducerMetrics("default_kafka", mockMetricRegistry);
+ }
+
+ @Test
+ public void incSuccessCounter() {
+ kafkaProducerMetrics.incSuccessCounter();
+ verify(mockSuccessCounter).inc();
+ }
+
+ @Test
+ public void incFailureCounter() {
+ kafkaProducerMetrics.incFailureCounter();
+ verify(mockFailureCounter).inc();
+ }
+
+ @Test
+ public void incRequestCounter() {
+ kafkaProducerMetrics.incRequestCounter();
+ verify(mockRequestCounter).inc();
+ }
+
+ @Test
+ public void getTimer() {
+ assertEquals(mockTimer, kafkaProducerMetrics.getTimer());
+ }
+}
\ No newline at end of file
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipelineTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipelineTest.java
new file mode 100644
index 0000000..aa2631e
--- /dev/null
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/KafkaToKafkaPipelineTest.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.MetricRegistry;
+import com.codahale.metrics.Timer;
+import com.expedia.www.haystack.pipes.key.extractor.Record;
+import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.slf4j.Logger;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import static com.expedia.www.haystack.pipes.commons.test.TestConstantsAndCommonCode.*;
+import static java.util.Collections.singletonList;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
+
+@RunWith(MockitoJUnitRunner.class)
+public class KafkaToKafkaPipelineTest {
+
+ private static final String TOPIC = RANDOM.nextLong() + "TOPIC";
+ private static final String KEY = RANDOM.nextLong() + "KEY";
+ private static final String VALUE = RANDOM.nextLong() + "VALUE";
+ @Mock
+ Timer.Context mockTimer;
+ private KafkaToKafkaPipeline kafkaToKafkaPipeline;
+ @Mock
+ private MetricRegistry mockMetricRegistry;
+ @Mock
+ private Logger mockLogger;
+ @Mock
+ private Timer mockKafkaProducerTimer;
+ @Mock
+ private KafkaProducer mockKafkaProducer;
+ @Mock
+ private SpanKeyExtractor mockSpanKeyExtractor;
+ @Mock
+ private KafkaProducerMetrics mockKafkaProducerMetrics;
+
+ private Logger realLogger;
+
+ @Before
+ public void setUp() {
+ realLogger = KafkaToKafkaPipeline.logger;
+ whenForConstructor();
+ kafkaToKafkaPipeline = new KafkaToKafkaPipeline(
+ singletonList(mockSpanKeyExtractor),
+ singletonList(new KafkaProducerWrapper("mockKafkaProducer", "mock-Topic", mockKafkaProducer, mockKafkaProducerMetrics))
+ );
+ }
+
+ private void whenForConstructor() {
+ when(mockSpanKeyExtractor.getRecords(any())).thenReturn(Collections.EMPTY_LIST);
+ when(mockMetricRegistry.timer(anyString())).thenReturn(mockKafkaProducerTimer);
+ when(mockKafkaProducerTimer.time()).thenReturn(mockTimer);
+ when(mockKafkaProducerMetrics.getTimer()).thenReturn(mockKafkaProducerTimer);
+ }
+
+ @Test
+ public void testProduceToKafkaTopics() {
+ KafkaToKafkaPipeline.logger = mockLogger;
+ kafkaToKafkaPipeline.produceToKafkaTopics(mockKafkaProducer, singletonList("mock-Topic"), "mock-key", JSON_SPAN_STRING_WITH_FLATTENED_TAGS, mockKafkaProducerMetrics);
+ verify(mockLogger).debug(KafkaToKafkaPipeline.kafkaProducerMsg, "mock-Topic");
+ KafkaToKafkaPipeline.logger = realLogger;
+ }
+
+ @Test
+ public void testApplyWithNullMessage() {
+ KafkaToKafkaPipeline mockKafkaToKafkaPipeline = new KafkaToKafkaPipeline(
+ singletonList(mockSpanKeyExtractor),
+ singletonList(new KafkaProducerWrapper("defaultTopic", "mockKafkaProducer", mockKafkaProducer, mockKafkaProducerMetrics)));
+ when(mockSpanKeyExtractor.getRecords(FULLY_POPULATED_SPAN)).thenReturn(Collections.EMPTY_LIST);
+ KafkaToKafkaPipeline.logger = mockLogger;
+
+ mockKafkaToKafkaPipeline.apply(null, FULLY_POPULATED_SPAN);
+ verify(mockLogger).debug("Extractor skipped the span: {}", FULLY_POPULATED_SPAN);
+ KafkaToKafkaPipeline.logger = realLogger;
+ }
+
+ @Test
+ public void testProduceToKafkaTopicsWithException() {
+ KafkaToKafkaPipeline.logger = mockLogger;
+ Exception exception = new RuntimeException();
+ when(mockKafkaProducer.send(any(), any())).thenThrow(exception);
+ kafkaToKafkaPipeline.produceToKafkaTopics(mockKafkaProducer, singletonList("mock-Topic"), "mock-key", JSON_SPAN_STRING_WITH_FLATTENED_TAGS, mockKafkaProducerMetrics);
+ verify(mockLogger).error(String.format(KafkaToKafkaPipeline.ERROR_MSG, JSON_SPAN_STRING_WITH_FLATTENED_TAGS, null), exception);
+ KafkaToKafkaPipeline.logger = realLogger;
+ }
+
+ @Test
+ public void testProduceToKafkaTopicsForCallback() {
+ KafkaToKafkaPipeline.logger = mockLogger;
+ Exception exception = new RuntimeException();
+
+ when(mockKafkaProducer.send(any(), any())).thenThrow(exception);
+ kafkaToKafkaPipeline.produceToKafkaTopics(mockKafkaProducer, singletonList("mock-Topic"), "mock-key", JSON_SPAN_STRING_WITH_FLATTENED_TAGS, mockKafkaProducerMetrics);
+
+ verify(mockLogger, times(1)).error(String.format(KafkaToKafkaPipeline.ERROR_MSG, JSON_SPAN_STRING_WITH_FLATTENED_TAGS, null), exception);
+ KafkaToKafkaPipeline.logger = realLogger;
+ }
+
+ @Test
+ public void testApplyWithTags() {
+ Logger realLogger = KafkaToKafkaPipeline.logger;
+ Record record = new Record(JSON_SPAN_STRING, "externalKafkaKey",
+ Collections.singletonMap("mock-Topic", singletonList("extractedTopic")));
+ when(mockSpanKeyExtractor.getRecords(any())).thenReturn(singletonList(record));
+ KafkaToKafkaPipeline.logger = mockLogger;
+ kafkaToKafkaPipeline.apply(null, FULLY_POPULATED_SPAN);
+ KafkaToKafkaPipeline.logger = realLogger;
+ verify(mockLogger).debug("Kafka Producer sending message: {},with key: {} ", JSON_SPAN_STRING_WITH_FLATTENED_TAGS, "externalKafkaKey");
+ }
+
+ @Test
+ public void testApplyWithoutTags() {
+ Logger realLogger = KafkaToKafkaPipeline.logger;
+ KafkaToKafkaPipeline.logger = mockLogger;
+ Record record = new Record(JSON_SPAN_STRING_WITH_NO_TAGS, "externalKafkaKey",
+ Collections.singletonMap("mock-Topic", Arrays.asList("extractedTopic")));
+ when(mockSpanKeyExtractor.getRecords(any())).thenReturn(singletonList(record));
+ kafkaToKafkaPipeline.apply(null, NO_TAGS_SPAN);
+ KafkaToKafkaPipeline.logger = realLogger;
+ verify(mockLogger).debug("Kafka Producer sending message: {},with key: {} ", JSON_SPAN_STRING_WITH_NO_TAGS, "externalKafkaKey");
+ }
+
+ @Test
+ public void testIfProducerTopicMappingIsNotPresent() {
+ Logger realLogger = KafkaToKafkaPipeline.logger;
+ Record record = new Record(JSON_SPAN_STRING, "externalKafkaKey",
+ Collections.singletonMap("DefaultKafkaProducer-mock", singletonList("extractedTopic")));
+ when(mockSpanKeyExtractor.getRecords(any())).thenReturn(singletonList(record));
+ KafkaToKafkaPipeline.logger = mockLogger;
+ kafkaToKafkaPipeline.apply(null, FULLY_POPULATED_SPAN);
+ KafkaToKafkaPipeline.logger = realLogger;
+ verify(mockLogger).error("Extractor skipped the span: {}, as no topics found for producer: {}", FULLY_POPULATED_SPAN, "mock-Topic");
+ }
+
+
+ @Test
+ public void testIfProducerTopicMappingIsNull() {
+ Logger realLogger = KafkaToKafkaPipeline.logger;
+ Record record = new Record(JSON_SPAN_STRING, "externalKafkaKey", null);
+ when(mockSpanKeyExtractor.getRecords(any())).thenReturn(singletonList(record));
+ KafkaToKafkaPipeline.logger = mockLogger;
+ kafkaToKafkaPipeline.apply(null, FULLY_POPULATED_SPAN);
+ KafkaToKafkaPipeline.logger = realLogger;
+ verify(mockLogger).error("Extractor skipped the span: {}, as no topics found for producer: {}", FULLY_POPULATED_SPAN, "mock-Topic");
+ }
+
+ @Test
+ public void testFactoryCreateProducerRecord() {
+ final ProducerRecord producerRecord = KafkaToKafkaPipeline.factory.createProducerRecord(TOPIC, KEY, VALUE);
+ assertEquals(KEY, producerRecord.key());
+ assertEquals(VALUE, producerRecord.value());
+ }
+}
\ No newline at end of file
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfigurationTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfigurationTest.java
new file mode 100644
index 0000000..d33d295
--- /dev/null
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProjectConfigurationTest.java
@@ -0,0 +1,100 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.expedia.www.haystack.pipes.commons.kafka.config.KafkaConsumerConfig;
+import com.expedia.www.haystack.pipes.kafka.producer.config.KafkaProducerConfig;
+import com.typesafe.config.Config;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ProjectConfigurationTest {
+
+ private ProjectConfiguration projectConfiguration;
+
+ @Before
+ public void setUp() {
+ projectConfiguration = ProjectConfiguration.getInstance();
+ }
+
+ @Test
+ public void testGetInstance() {
+ ProjectConfiguration newProjectConfiguration = ProjectConfiguration.getInstance();
+ assertEquals(projectConfiguration, newProjectConfiguration);
+ ProjectConfiguration.projectConfiguration = null;
+ assertNotEquals(ProjectConfiguration.getInstance(), null);
+ }
+
+ @Test
+ public void testGetKafkaConsumerConfig() {
+ KafkaConsumerConfig kafkaConsumerConfig = projectConfiguration.getKafkaConsumerConfig();
+ assertEquals("localhost", kafkaConsumerConfig.brokers());
+ assertEquals(9092, kafkaConsumerConfig.port());
+ assertEquals("json-spans", kafkaConsumerConfig.fromtopic());
+ assertEquals(1, kafkaConsumerConfig.threadcount());
+ assertEquals(15000, kafkaConsumerConfig.sessiontimeout());
+ assertEquals(10, kafkaConsumerConfig.maxwakeups());
+ assertEquals(3000, kafkaConsumerConfig.wakeuptimeoutms());
+ assertEquals(250, kafkaConsumerConfig.polltimeoutms());
+ assertEquals(3000, kafkaConsumerConfig.commitms());
+ }
+
+ @Test
+ public void testGetKafkaProducerConfigList() {
+ List kafkaProducerConfigs = projectConfiguration.getKafkaProducerConfigs();
+ assertEquals(1, kafkaProducerConfigs.size());
+ KafkaProducerConfig kafkaProducerConfig = kafkaProducerConfigs.get(0);
+ assertEquals("localhost:9092", kafkaProducerConfig.getBrokers());
+ assertEquals(9093, kafkaProducerConfig.getPort());
+ assertEquals("externalKafkaTopic", kafkaProducerConfig.getDefaultTopic());
+ assertEquals("0", kafkaProducerConfig.getAcks());
+ assertEquals(8192, kafkaProducerConfig.getBatchSize());
+ assertEquals(4, kafkaProducerConfig.getLingerMs());
+ assertEquals(1024, kafkaProducerConfig.getBufferMemory());
+
+ }
+
+ @Test
+ public void testGetSpanExtractorConfigs() {
+ Map spanKeyExtractorConfigMap = projectConfiguration.getSpanExtractorConfigs();
+ assertEquals(1, spanKeyExtractorConfigMap.size());
+ }
+
+ @Test
+ public void testKafkaConsumerConfigIdempotent() {
+ assertEquals(projectConfiguration.getKafkaConsumerConfig(), projectConfiguration.getKafkaConsumerConfig());
+ }
+
+ @Test
+ public void testKafkaProducerConfigsIdempotent() {
+ assertEquals(projectConfiguration.getKafkaProducerConfigs(), projectConfiguration.getKafkaProducerConfigs());
+ }
+
+ @Test
+ public void testSpanKeyExtractorConfigsIdempotent() {
+ assertEquals(projectConfiguration.getSpanExtractorConfigs(), projectConfiguration.getSpanExtractorConfigs());
+ }
+}
\ No newline at end of file
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducerTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducerTest.java
similarity index 62%
rename from kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducerTest.java
rename to kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducerTest.java
index 2b6b623..3d4e3d3 100644
--- a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ProtobufToKafkaProducerTest.java
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ProtobufToKafkaProducerTest.java
@@ -1,5 +1,5 @@
/*
- * Copyright 2018 Expedia, Inc.
+ * Copyright 2020 Expedia, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -14,11 +14,11 @@
* limitations under the License.
*
*/
-package com.expedia.www.haystack.pipes.kafkaProducer;
+package com.expedia.www.haystack.pipes.kafka.producer;
import com.expedia.open.tracing.Span;
-import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
+import com.expedia.www.haystack.pipes.commons.kafka.config.KafkaConsumerConfig;
import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
import org.apache.kafka.common.serialization.Serde;
import org.apache.kafka.common.serialization.Serdes;
@@ -33,13 +33,8 @@
import org.mockito.runners.MockitoJUnitRunner;
import static com.expedia.www.haystack.pipes.commons.test.TestConstantsAndCommonCode.RANDOM;
-import static com.expedia.www.haystack.pipes.kafkaProducer.Constants.APPLICATION;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
@RunWith(MockitoJUnitRunner.class)
public class ProtobufToKafkaProducerTest {
@@ -50,9 +45,9 @@ public class ProtobufToKafkaProducerTest {
@Mock
private SerdeFactory mockSerdeFactory;
@Mock
- private ProduceIntoExternalKafkaAction mockProduceIntoExternalKafkaAction;
+ private KafkaToKafkaPipeline mockKafkaToKafkaPipeline;
@Mock
- private KafkaConfigurationProvider mockKafkaConfigurationProvider;
+ private KafkaConsumerConfig mockKafkaConsumerConfig;
@Mock
private KStreamBuilder mockKStreamBuilder;
@Mock
@@ -60,40 +55,40 @@ public class ProtobufToKafkaProducerTest {
@Mock
private Serde mockSpanSerde;
- private ProtobufToKafkaProducer protobufToFirehoseProducer;
+ private ProtobufToKafkaProducer protobufToKafkaProducer;
@Before
public void setUp() {
- protobufToFirehoseProducer = new ProtobufToKafkaProducer(
- mockKafkaStreamStarter, mockSerdeFactory, mockProduceIntoExternalKafkaAction, mockKafkaConfigurationProvider);
+ protobufToKafkaProducer = new ProtobufToKafkaProducer(
+ mockKafkaStreamStarter, mockSerdeFactory, mockKafkaToKafkaPipeline, mockKafkaConsumerConfig);
}
@After
public void tearDown() {
- verifyNoMoreInteractions(mockKafkaStreamStarter, mockSerdeFactory, mockProduceIntoExternalKafkaAction,
- mockKafkaConfigurationProvider, mockKStreamBuilder, mockKStream, mockSpanSerde);
+ verifyNoMoreInteractions(mockKafkaStreamStarter, mockSerdeFactory, mockKafkaToKafkaPipeline,
+ mockKafkaConsumerConfig, mockKStreamBuilder, mockKStream, mockSpanSerde);
}
@Test
public void testMain() {
- protobufToFirehoseProducer.main();
+ protobufToKafkaProducer.main();
- verify(mockKafkaStreamStarter).createAndStartStream(protobufToFirehoseProducer);
+ verify(mockKafkaStreamStarter).createAndStartStream(protobufToKafkaProducer);
}
@SuppressWarnings("Duplicates")
@Test
public void testBuildStreamTopology() {
when(mockSerdeFactory.createJsonProtoSpanSerde(anyString())).thenReturn(mockSpanSerde);
- when(mockKafkaConfigurationProvider.fromtopic()).thenReturn(FROM_TOPIC);
+ when(mockKafkaConsumerConfig.fromtopic()).thenReturn(FROM_TOPIC);
when(mockKStreamBuilder.stream(Matchers.>any(), Matchers.>any(), anyString()))
.thenReturn(mockKStream);
- protobufToFirehoseProducer.buildStreamTopology(mockKStreamBuilder);
+ protobufToKafkaProducer.buildStreamTopology(mockKStreamBuilder);
- verify(mockSerdeFactory).createJsonProtoSpanSerde(APPLICATION);
- verify(mockKafkaConfigurationProvider).fromtopic();
+ verify(mockSerdeFactory).createJsonProtoSpanSerde(Constants.APPLICATION);
+ verify(mockKafkaConsumerConfig).fromtopic();
verify(mockKStreamBuilder).stream(any(Serdes.StringSerde.class), eq(mockSpanSerde), eq(FROM_TOPIC));
- verify(mockKStream).foreach(mockProduceIntoExternalKafkaAction);
+ verify(mockKStream).foreach(mockKafkaToKafkaPipeline);
}
}
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ServiceTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ServiceTest.java
new file mode 100644
index 0000000..1ef3fdc
--- /dev/null
+++ b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafka/producer/ServiceTest.java
@@ -0,0 +1,108 @@
+package com.expedia.www.haystack.pipes.kafka.producer;
+
+import com.codahale.metrics.JmxReporter;
+import com.expedia.www.haystack.pipes.commons.health.HealthController;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.slf4j.Logger;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ServiceTest {
+
+ @Mock
+ private Logger mockLogger;
+ @Mock
+ private HealthController mockHealthController;
+ @Mock
+ private ProtobufToKafkaProducer mockProtobufToKafkaProducer;
+ @Mock
+ private Service mockService;
+ @Mock
+ private JmxReporter mockJmxReporter;
+
+ private Service service;
+
+ @Before
+ public void setUp() {
+ service = Service.getInstance();
+ }
+
+ @Test
+ public void testMain() {
+ Logger realLogger = Service.logger;
+ Service.logger = mockLogger;
+ Service realService = Service.service;
+ Service.service = mockService;
+ when(mockService.getProtobufToKafkaProducer(any())).thenReturn(mockProtobufToKafkaProducer);
+ when(mockService.getJmxReporter()).thenReturn(mockJmxReporter);
+ Service.main(new String[0]);
+ Service.service = realService;
+ Service.logger = realLogger;
+ verify(mockLogger).info("Initializing Kafka Consumers");
+ verify(mockProtobufToKafkaProducer).main();
+ }
+
+ @Test
+ public void testGetInstance() {
+ Service service = Service.getInstance();
+ assertEquals(service, Service.getInstance());
+ }
+
+
+ @Test
+ public void testGetKafkaStreamStarter() {
+ KafkaStreamStarter kafkaStreamStarter = service.getKafkaStreamStarter();
+ assertEquals(ProtobufToKafkaProducer.class, kafkaStreamStarter.containingClass);
+ }
+
+ @Test
+ public void testInPlaceHealthCheck() {
+ HealthController realHealthController = Service.healthController;
+ Service.healthController = mockHealthController;
+ service.inPlaceHealthCheck();
+ Service.healthController = realHealthController;
+ verify(mockHealthController).addListener(any());
+ }
+
+ @Test
+ public void testGetJmxReporter() {
+ assertEquals(JmxReporter.class, service.getJmxReporter().getClass());
+ }
+
+ @Test
+ public void testGetExtractorKafkaProducerMap() {
+ ProjectConfiguration projectConfiguration = ProjectConfiguration.getInstance();
+ List kafkaProducerWrappers = Service.getKafkaProducerWrappers(projectConfiguration);
+ assertEquals(kafkaProducerWrappers.size(), 1);
+ }
+
+ @Test
+ public void testGetExtractorKafkaProducerMapForIdempotent() {
+ ProjectConfiguration projectConfiguration = ProjectConfiguration.getInstance();
+ List kafkaProducerWrappers = Service.getKafkaProducerWrappers(projectConfiguration);
+ assertEquals(kafkaProducerWrappers, Service.getKafkaProducerWrappers(projectConfiguration));
+ }
+
+ @Test
+ public void testGetKafkaToKafkaPipeline() {
+ assertEquals(KafkaToKafkaPipeline.class, service.getKafkaToKafkaPipeline().getClass());
+ }
+
+ @Test
+ public void testGetProtobufToKafkaProducer() {
+ assertEquals(ProtobufToKafkaProducer.class, service.getProtobufToKafkaProducer(service.getKafkaStreamStarter()).getClass());
+ }
+
+}
+
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProviderTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProviderTest.java
deleted file mode 100644
index ce73503..0000000
--- a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/ExternalKafkaConfigurationProviderTest.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-
-public class ExternalKafkaConfigurationProviderTest {
- private ExternalKafkaConfigurationProvider externalKafkaConfigurationProvider;
-
- @Before
- public void setUp() {
- externalKafkaConfigurationProvider = new ExternalKafkaConfigurationProvider();
- }
-
- @Test
- public void testBrokers() {
- assertEquals("localhost:9092", externalKafkaConfigurationProvider.brokers());
- }
-
- @Test
- public void testPort() {
- assertEquals(9093, externalKafkaConfigurationProvider.port());
- }
-
- @Test
- public void testToTopic() {
- assertEquals("externalKafkaTopic", externalKafkaConfigurationProvider.totopic());
- }
-
- @Test
- public void testAcks() {
- assertEquals("0", externalKafkaConfigurationProvider.acks());
- }
-
- @Test
- public void testBatchSize() {
- assertEquals(8192, externalKafkaConfigurationProvider.batchsize());
- }
-
- @Test
- public void testLingerMs() {
- assertEquals(4, externalKafkaConfigurationProvider.lingerms());
- }
-
- @Test
- public void testBufferMemory() {
- assertEquals(1024, externalKafkaConfigurationProvider.buffermemory());
- }
-
- @Test
- public void testReload() {
- externalKafkaConfigurationProvider.reload();
- testBrokers();
- testPort();
- testToTopic();
- testAcks();
- testBatchSize();
- testLingerMs();
- testBufferMemory();
- }
-}
diff --git a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveControllerTest.java b/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveControllerTest.java
deleted file mode 100644
index ce53f46..0000000
--- a/kafka-producer/src/test/java/com/expedia/www/haystack/pipes/kafkaProducer/KafkaProducerIsActiveControllerTest.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Copyright 2018 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer;
-
-import com.expedia.www.haystack.pipes.kafkaProducer.KafkaProducerIsActiveController.Factory;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.mockito.Mock;
-import org.mockito.runners.MockitoJUnitRunner;
-import org.slf4j.Logger;
-import org.springframework.boot.SpringApplication;
-
-import java.util.Set;
-
-import static com.expedia.www.haystack.pipes.kafkaProducer.KafkaProducerIsActiveController.STARTUP_MSG;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertSame;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
-
-@RunWith(MockitoJUnitRunner.class)
-public class KafkaProducerIsActiveControllerTest {
- @Mock
- private Factory mockFactory;
-
- @Mock
- private ProtobufToKafkaProducer mockProtobufToKafkaProducer;
-
- @Mock
- private SpringApplication mockSpringApplication;
-
- @Mock
- private Logger mockLogger;
-
- private Factory factory;
-
- @Before
- public void setUp() {
- storeKafkaProducerIsActiveControllerWithMocksInStaticInstance();
- factory = new Factory();
- }
-
- private void storeKafkaProducerIsActiveControllerWithMocksInStaticInstance() {
- new KafkaProducerIsActiveController(mockProtobufToKafkaProducer, mockFactory, mockLogger);
- }
-
- @After
- public void tearDown() {
- verifyNoMoreInteractions(mockProtobufToKafkaProducer, mockFactory, mockSpringApplication, mockLogger);
- clearKafkaProducerIsActiveControllerInStaticInstance();
- }
-
- private void clearKafkaProducerIsActiveControllerInStaticInstance() {
- KafkaProducerIsActiveController.INSTANCE.set(null);
- }
-
- @Test
- public void testMain() {
- when(mockFactory.createSpringApplication()).thenReturn(mockSpringApplication);
-
- final String[] args = new String[0];
- KafkaProducerIsActiveController.main(args);
-
- verify(mockLogger).info(STARTUP_MSG);
- verify(mockFactory).createSpringApplication();
- verify(mockProtobufToKafkaProducer).main();
- verify(mockSpringApplication).run(args);
- }
-
- @Test
- public void testFactoryCreateSpringApplication() {
- final SpringApplication springApplication = factory.createSpringApplication();
-
- final Set
@@ -159,7 +160,7 @@
com.expedia.www
haystack-pipes-commons
- ${haystack-pipes-commons.version}
+ ${project.version}
com.fasterxml.jackson.core
@@ -314,7 +315,16 @@
spring-context
${spring-context.version}
-
+
+ com.expedia.www
+ haystack-commons
+ ${haystack-commons.version}
+
+
+ com.typesafe
+ config
+ ${typesafe-config.version}
+
junit
@@ -380,6 +390,8 @@
**/com/expedia/open/tracing/**
**/com/expedia/www/haystack/pipes/commons/test/*
**/com/expedia/www/haystack/pipes/secretDetector/actions/SenderImpl*
+ **/com/expedia/www/haystack/pipes/kafka/producer/Constants*
+ **/com/expedia/www/haystack/pipes/kafka/producer/KafkaProducerWrapper*
diff --git a/sample-key-extractor/Makefile b/sample-key-extractor/Makefile
new file mode 100644
index 0000000..6b531b9
--- /dev/null
+++ b/sample-key-extractor/Makefile
@@ -0,0 +1,10 @@
+.PHONY: release
+
+export DOCKER_ORG := expediadotcom
+export DOCKER_IMAGE_NAME := haystack-pipes-kafka-producer-extractor
+
+docker_build:
+ docker build -t $(DOCKER_IMAGE_NAME) -f build/docker/Dockerfile .
+
+release: docker_build
+ ../deployment/scripts/publish-to-docker-hub.sh
diff --git a/sample-key-extractor/README.md b/sample-key-extractor/README.md
new file mode 100644
index 0000000..f0c5bb5
--- /dev/null
+++ b/sample-key-extractor/README.md
@@ -0,0 +1,5 @@
+##Sample SpanKey Extractor
+
+It is a sample project that use java SPI to use [Sample Extractor](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java) as extractor that transforms span to json.
+[Dockerfile](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/sample-key-extractor/build/docker/Dockerfile) helps to deploy this project. The configuration required by the extractor is already mentioned in [base.conf](https://github.com/ExpediaDotCom/haystack-pipes/blob/master/kafka-producer/src/main/resources/config/base.conf).
+
diff --git a/sample-key-extractor/build/docker/Dockerfile b/sample-key-extractor/build/docker/Dockerfile
index fdc1e63..dfc875a 100644
--- a/sample-key-extractor/build/docker/Dockerfile
+++ b/sample-key-extractor/build/docker/Dockerfile
@@ -1,14 +1,8 @@
FROM haystack-pipes-kafka-producer:latest
MAINTAINER Haystack
-
RUN mkdir -p ${APP_HOME}/extractors/
-COPY target/haystack-sample-key-extractors.jar ${APP_HOME}/extractors/
-#COPY build/docker/extractors.conf ${APP_HOME}/
-#ENV EXTRACTOR_CONFIG_PATH ${APP_HOME}/extractors.conf
+COPY target/sample-key-extractor.jar ${APP_HOME}/extractors/
RUN ls ${APP_HOME}/extractors/
-
-
-
WORKDIR ${APP_HOME}
diff --git a/sample-key-extractor/pom.xml b/sample-key-extractor/pom.xml
index 0c31c40..f60684f 100644
--- a/sample-key-extractor/pom.xml
+++ b/sample-key-extractor/pom.xml
@@ -5,9 +5,10 @@
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
4.0.0
+ 2.0.0-SNAPSHOT
sample-key-extractor
diff --git a/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/extractor/SampleExtractor.java b/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/extractor/SampleExtractor.java
new file mode 100644
index 0000000..a68ba18
--- /dev/null
+++ b/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafka/producer/extractor/SampleExtractor.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.kafka.producer.extractor;
+
+import com.expedia.open.tracing.Span;
+import com.expedia.www.haystack.pipes.key.extractor.Record;
+import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
+import com.google.protobuf.InvalidProtocolBufferException;
+import com.google.protobuf.util.JsonFormat;
+import com.netflix.servo.util.VisibleForTesting;
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public class SampleExtractor implements SpanKeyExtractor {
+
+ @VisibleForTesting
+ static Logger logger = LoggerFactory.getLogger(SampleExtractor.class);
+ @VisibleForTesting
+ static JsonFormat.Printer jsonPrinter = JsonFormat.printer().omittingInsignificantWhitespace();
+ private final String key = "sampleKafkaKey";
+ private List producers;
+ private Map> producerTopicsMapping;
+
+ @Override
+ public String name() {
+ return "SampleExtractor";
+ }
+
+ @Override
+ public void configure(Config config) {
+ logger.info("{} class loaded with config: {}", SampleExtractor.class.getSimpleName(), config);
+ List producerConf = (List) config.getConfigList("producers");
+ // populating producer topic mapping from configuration
+ producerTopicsMapping = producerConf.stream().collect(Collectors.toMap(conf -> conf.getString("name"),
+ conf -> conf.getConfig("config").getStringList("topics")));
+ }
+
+ @Override
+ public List getRecords(Span span) {
+ try {
+ Map> producerTopicsMapping = new HashMap<>();
+ Record record = new Record(jsonPrinter.print(span), key, producerTopicsMapping);
+ return Collections.singletonList(record);
+ } catch (InvalidProtocolBufferException e) {
+ logger.error("Exception occurred while extracting span with traceId:{} {}", span.getTraceId(), e.getMessage());
+ }
+ return Collections.emptyList();
+ }
+
+}
diff --git a/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java b/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java
deleted file mode 100644
index c129f67..0000000
--- a/sample-key-extractor/src/main/java/com/expedia/www/haystack/pipes/kafkaProducer/extractor/SampleExtractor.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2020 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.kafkaProducer.extractor;
-
-import com.expedia.open.tracing.Span;
-import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
-import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.util.JsonFormat;
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Optional;
-
-public class SampleExtractor implements SpanKeyExtractor {
-
- private final JsonFormat.Printer jsonPrinter = JsonFormat.printer();
- private static final Logger logger = LoggerFactory.getLogger("SampleExtractor");
-
- @Override
- public String name() {
- return "SampleExtractor";
- }
-
- @Override
- public void configure(Config config) {
- logger.debug("{} got config: {}", name(), config);
- }
-
- @Override
- public Optional extract(Span span) {
- try {
- return Optional.of(jsonPrinter.print(span));
- } catch (InvalidProtocolBufferException e) {
- logger.error("Exception occurred while extracting span: " + e.getMessage());
- }
- return Optional.empty();
- }
-
- @Override
- public String getKey() {
- return "dummy-key";
- }
-
- @Override
- public List getTopics() {
- return new ArrayList<>();
- }
-
-}
diff --git a/sample-key-extractor/src/main/resources/META-INF/services/com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor b/sample-key-extractor/src/main/resources/META-INF/services/com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor
index d4f7bc0..1c10d2b 100644
--- a/sample-key-extractor/src/main/resources/META-INF/services/com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor
+++ b/sample-key-extractor/src/main/resources/META-INF/services/com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor
@@ -1 +1 @@
-com.expedia.www.haystack.pipes.kafkaProducer.extractor.SampleExtractor
\ No newline at end of file
+com.expedia.www.haystack.pipes.kafka.producer.extractor.SampleExtractor
\ No newline at end of file
diff --git a/secret-detector/pom.xml b/secret-detector/pom.xml
index ed925d8..1bde24d 100644
--- a/secret-detector/pom.xml
+++ b/secret-detector/pom.xml
@@ -6,14 +6,14 @@
haystack-pipes-secret-detector
jar
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
haystack-pipes-secret-detector
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
diff --git a/secret-detector/src/main/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfig.java b/secret-detector/src/main/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfig.java
index cea1f4a..ec20e04 100644
--- a/secret-detector/src/main/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfig.java
+++ b/secret-detector/src/main/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfig.java
@@ -33,17 +33,13 @@
import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
import com.expedia.www.haystack.pipes.commons.serialization.SerdeFactory;
-import com.expedia.www.haystack.pipes.secretDetector.actions.EmailerDetectedAction;
-import com.expedia.www.haystack.pipes.secretDetector.actions.EmailerDetectedActionFactory;
-import com.expedia.www.haystack.pipes.secretDetector.actions.SenderImpl;
+import com.expedia.www.haystack.pipes.secretDetector.actions.*;
import com.expedia.www.haystack.pipes.secretDetector.config.ActionsConfigurationProvider;
import com.expedia.www.haystack.pipes.secretDetector.config.SecretsEmailConfigurationProvider;
import com.expedia.www.haystack.pipes.secretDetector.config.SpringWiredWhiteListConfigurationProvider;
import com.expedia.www.haystack.pipes.secretDetector.mains.ProtobufSpanMaskerToKafkaTransformer;
import com.expedia.www.haystack.pipes.secretDetector.mains.ProtobufSpanToEmailInKafkaTransformer;
import com.expedia.www.haystack.pipes.secretDetector.mains.ProtobufToDetectorAction;
-import com.expedia.www.haystack.pipes.secretDetector.actions.FromAddressExceptionLogger;
-import com.expedia.www.haystack.pipes.secretDetector.actions.ToAddressExceptionLogger;
import com.netflix.servo.monitor.Counter;
import com.netflix.servo.monitor.Timer;
import org.cfg4j.provider.ConfigurationProvider;
@@ -162,8 +158,8 @@ SpanS3ConfigFetcher.SpanFactory s3ConfigFetcherFactory() {
@Bean
@Autowired
- KafkaStreamStarter kafkaStreamStarter(final HealthController healthController) {
- return new KafkaStreamStarter(ProtobufToDetectorAction.class, APPLICATION, healthController);
+ KafkaStreamStarter kafkaStreamStarter(final HealthController healthController, KafkaConfigurationProvider kafkaConfigurationProvider) {
+ return new KafkaStreamStarter(ProtobufToDetectorAction.class, APPLICATION, kafkaConfigurationProvider, healthController);
}
@Bean
diff --git a/secret-detector/src/main/resources/base.yaml b/secret-detector/src/main/resources/base.yaml
index 7200b59..bf156d2 100644
--- a/secret-detector/src/main/resources/base.yaml
+++ b/secret-detector/src/main/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: -1
polltimeoutms: -1
commitms: -1
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "localhost"
diff --git a/secret-detector/src/test/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfigTest.java b/secret-detector/src/test/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfigTest.java
index 0ecbc80..24a3ddc 100644
--- a/secret-detector/src/test/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfigTest.java
+++ b/secret-detector/src/test/java/com/expedia/www/haystack/pipes/secretDetector/SpringConfigTest.java
@@ -25,6 +25,7 @@
import com.expedia.www.haystack.pipes.commons.TimersAndCounters;
import com.expedia.www.haystack.pipes.commons.health.HealthController;
import com.expedia.www.haystack.pipes.commons.health.HealthStatusListener;
+import com.expedia.www.haystack.pipes.commons.kafka.KafkaConfigurationProvider;
import com.expedia.www.haystack.pipes.commons.kafka.KafkaStreamStarter;
import com.expedia.www.haystack.pipes.secretDetector.actions.EmailerDetectedAction;
import com.expedia.www.haystack.pipes.secretDetector.actions.FromAddressExceptionLogger;
@@ -50,14 +51,10 @@
import static com.expedia.www.haystack.pipes.secretDetector.Constants.APPLICATION;
import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertSame;
+import static org.junit.Assert.*;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.*;
@RunWith(MockitoJUnitRunner.class)
public class SpringConfigTest {
@@ -89,6 +86,8 @@ public class SpringConfigTest {
private Clock mockClock;
@Mock
private Timer mockSpanArrivalTimer;
+ @Mock
+ private KafkaConfigurationProvider mockKafkaConfigurationProvider;
private Timers timers;
private SpringConfig springConfig;
@@ -175,7 +174,7 @@ public void testSpanNameAndCountRecorderLogger() {
@Test
public void testKafkaStreamStarter() {
- final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController);
+ final KafkaStreamStarter kafkaStreamStarter = springConfig.kafkaStreamStarter(mockHealthController, mockKafkaConfigurationProvider);
assertSame(ProtobufToDetectorAction.class, kafkaStreamStarter.containingClass);
assertSame(APPLICATION, kafkaStreamStarter.clientId);
diff --git a/secret-detector/src/test/resources/base.yaml b/secret-detector/src/test/resources/base.yaml
index 2a708df..5f5bd84 100644
--- a/secret-detector/src/test/resources/base.yaml
+++ b/secret-detector/src/test/resources/base.yaml
@@ -10,9 +10,6 @@ haystack:
wakeuptimeoutms: -1
polltimeoutms: -1
commitms: -1
- pipe:
- streams:
- replicationfactor: 1
graphite:
prefix: "haystack"
host: "localhost"
diff --git a/span-key-extractor/README.md b/span-key-extractor/README.md
new file mode 100644
index 0000000..619e2e1
--- /dev/null
+++ b/span-key-extractor/README.md
@@ -0,0 +1,8 @@
+### Span Key Extractor
+
+It uses Java SPI model to load different key-extractors
+for each module in given project. It reads all the span-key extractors jars
+from the directory: `extractors/`
+
+A sample project is also added to haystack-pipes to explain in detail:
+[sample-key-extractor](https://github.com/ExpediaDotCom/haystack-pipes/tree/master/sample-key-extractor)
diff --git a/span-key-extractor/pom.xml b/span-key-extractor/pom.xml
index 7e2394e..77e278d 100644
--- a/span-key-extractor/pom.xml
+++ b/span-key-extractor/pom.xml
@@ -5,23 +5,22 @@
haystack-pipes
com.expedia.www
- 1.0-SNAPSHOT
+ 2.0.0-SNAPSHOT
4.0.0
jar
span-key-extractor
+ 2.0.0-SNAPSHOT
com.expedia.www
haystack-commons
- 1.0.50
com.typesafe
config
- 1.3.1
org.cfg4j
diff --git a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/ProjectConfiguration.java b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/ProjectConfiguration.java
deleted file mode 100644
index 569cc09..0000000
--- a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/ProjectConfiguration.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2020 Expedia, Inc.
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-package com.expedia.www.haystack.pipes.key.extractor;
-
-import com.expedia.www.haystack.commons.config.ConfigurationLoader;
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-
-public class ProjectConfiguration {
-
- private static final Logger logger = LoggerFactory.getLogger(ProjectConfiguration.class);
-
- private final Config haystackConf;
- private final String directory;
-
- public ProjectConfiguration() {
- Config config = ConfigurationLoader.loadConfigFileWithEnvOverrides("config/base.conf", Optional.empty().toString());
- logger.debug("loaded config: {}", config);
- haystackConf = config.getConfig(Constants.HAYSTACK_KEY);
- directory = haystackConf.getString(Constants.DIRECTORY_KEY);
-
- }
-
- public Map getSpanExtractorConfigs() {
- Map extractorConfigMap = new HashMap<>();
- if (haystackConf != null) {
- List extractorConfigs = (List) haystackConf.getConfigList(Constants.EXTRACTORS_KEY);
- extractorConfigs.forEach(extractorConfig -> {
- String name = extractorConfig.getString(Constants.EXTRACTOR_NAME_KEY);
- Config config = extractorConfig.getConfig(Constants.EXTRACTOR_CONFIG_KEY);
- extractorConfigMap.put(name, config);
- });
- }
- return extractorConfigMap;
- }
-
- public String getDirectory() {
- return directory;
- }
-}
diff --git a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Record.java b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Record.java
new file mode 100644
index 0000000..ca0d438
--- /dev/null
+++ b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/Record.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2020 Expedia, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package com.expedia.www.haystack.pipes.key.extractor;
+
+import java.util.List;
+import java.util.Map;
+
+public class Record {
+
+ String message; // message after span-key extractor manipulation
+ String key; // key after span-key extractor manipulation
+ Map> producerTopicsMappings; // producer with list of topics after span-key extractor manipulation
+
+ public Record(String message, String key, Map> producerTopicsMappings) {
+ this.message = message;
+ this.key = key;
+ this.producerTopicsMappings = producerTopicsMappings;
+ }
+
+ public String getMessage() {
+ return message;
+ }
+
+ public Map> getProducerTopicsMappings() {
+ return producerTopicsMappings;
+ }
+
+ public String getKey() {
+ return key;
+ }
+}
diff --git a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/SpanKeyExtractor.java b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/SpanKeyExtractor.java
index 434801b..427901d 100644
--- a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/SpanKeyExtractor.java
+++ b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/SpanKeyExtractor.java
@@ -20,18 +20,13 @@
import com.typesafe.config.Config;
import java.util.List;
-import java.util.Optional;
public interface SpanKeyExtractor {
String name();
- public void configure(Config config);
+ public void configure(Config config); // sets up the extractor with configuration
- public Optional extract(Span span);
-
- public String getKey();
-
- public List getTopics();
+ public List getRecords(Span span); // returns list of records containing message, key and producer topic mapping
}
diff --git a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/loader/SpanKeyExtractorLoader.java b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/loader/SpanKeyExtractorLoader.java
index 925071e..1a3a5b5 100644
--- a/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/loader/SpanKeyExtractorLoader.java
+++ b/span-key-extractor/src/main/java/com/expedia/www/haystack/pipes/key/extractor/loader/SpanKeyExtractorLoader.java
@@ -16,8 +16,8 @@
*/
package com.expedia.www.haystack.pipes.key.extractor.loader;
-import com.expedia.www.haystack.pipes.key.extractor.ProjectConfiguration;
import com.expedia.www.haystack.pipes.key.extractor.SpanKeyExtractor;
+import com.typesafe.config.Config;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -26,19 +26,18 @@
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.List;
+import java.util.Map;
import java.util.ServiceLoader;
public class SpanKeyExtractorLoader {
+ private static final Logger logger = LoggerFactory.getLogger(SpanKeyExtractorLoader.class);
private static SpanKeyExtractorLoader spanKeyExtractorLoader = null;
- private static final Logger logger = LoggerFactory.getLogger("SpanKeyExtractorLoader");
- private ProjectConfiguration projectConfiguration;
private List spanKeyExtractorList;
private ServiceLoader serviceLoader;
private SpanKeyExtractorLoader() {
- projectConfiguration = new ProjectConfiguration();
spanKeyExtractorList = new ArrayList<>();
}
@@ -52,7 +51,7 @@ public static synchronized SpanKeyExtractorLoader getInstance() {
private void loadFiles() {
try {
- final File[] extractorFile = new File(projectConfiguration.getDirectory()).listFiles();
+ final File[] extractorFile = new File("extractors/").listFiles();
if (extractorFile != null) {
final List urls = new ArrayList<>();
for (final File file : extractorFile) {
@@ -68,14 +67,13 @@ private void loadFiles() {
}
}
- public List getSpanKeyExtractor() {
+ public List getSpanKeyExtractor(Map spanKeyExtractorConfigs) {
if (spanKeyExtractorList.isEmpty() && this.serviceLoader != null) {
serviceLoader.forEach(spanKeyExtractor -> {
try {
- spanKeyExtractor.configure(projectConfiguration.getSpanExtractorConfigs()
- .getOrDefault(spanKeyExtractor.name(), null));
+ spanKeyExtractor.configure(spanKeyExtractorConfigs.getOrDefault(spanKeyExtractor.name(), null));
spanKeyExtractorList.add(spanKeyExtractor);
- logger.debug("Extractor class is loaded: {}, at path: {}", spanKeyExtractor.name(), projectConfiguration.getDirectory());
+ logger.debug("Extractor class is loaded: {}, at path: {}", spanKeyExtractor.name(), "extractors/");
} catch (Exception e) {
logger.error("Failed to load Span Extractor, Exception: {}", e.getMessage());
}