From b3e8fa4347bad134326f1bff22b48a9e4e673205 Mon Sep 17 00:00:00 2001 From: Jae Hyeon Bae Date: Fri, 6 Jun 2014 01:12:35 -0700 Subject: [PATCH] Create SuroInput - input.json file on conf directory, by default only thrift server input - addInputType on SuroPlugin - DefaultObjectMapper additional class injection - TrafficController registers/unregister input - KafkaConsumer implements SuroInput - ThriftServer implements SuroInput --- build.gradle | 4 +- .../java/com/netflix/suro/SuroPlugin.java | 9 ++ .../com/netflix/suro/input/SuroInput.java | 13 ++ .../suro/jackson/DefaultObjectMapper.java | 6 +- .../netflix/suro/queue/TrafficController.java | 31 +++- .../netflix/suro/routing/MessageRouter.java | 4 +- .../com/netflix/suro/sink/SinkManager.java | 5 +- .../java/com/netflix/suro/TestSuroServer.java | 8 +- .../suro/input/kafka/KafkaConsumer.java | 142 ++++++++++++++++++ .../suro/input/kafka/TestKafkaConsumer.java | 138 +++++++++++++++++ .../suro/sink/kafka/TestKafkaSink.java | 6 +- .../suro/sink/kafka/ZkExternalResource.java | 1 + .../suro/sink/localfile/LocalFileSink.java | 21 +-- .../sink/localfile/TestLocalFileSink.java | 84 +---------- .../suro/sink/remotefile/S3FileSink.java | 4 +- .../suro/sink/remotefile/TestS3FileSink.java | 18 +-- suro-server/conf/input.json | 5 + .../suro/SuroDynamicPropertyModule.java | 2 + .../java/com/netflix/suro/SuroModule.java | 14 +- .../java/com/netflix/suro/SuroServer.java | 13 ++ .../java/com/netflix/suro/SuroService.java | 20 +-- .../DynamicPropertyInputConfigurator.java | 55 +++++++ .../com/netflix/suro/input/InputManager.java | 52 +++++++ .../netflix/suro/input/SuroInputPlugin.java | 13 ++ .../thrift}/CustomServerSocket.java | 2 +- .../thrift}/MessageSetProcessor.java | 8 +- .../thrift}/MessageSetSerDe.java | 2 +- .../thrift}/ServerConfig.java | 2 +- .../thrift}/ThriftServer.java | 70 ++++++--- .../queue/MessageSetProcessorManager.java | 53 ------- .../com/netflix/suro/queue/Queue4Server.java | 5 +- .../com/netflix/suro/server/HealthCheck.java | 1 + .../com/netflix/suro/server/StatusServer.java | 3 +- .../netflix/suro/sink/notice/SQSNotice.java | 4 +- .../netflix/suro/input/TestInputManager.java | 128 ++++++++++++++++ .../thrift}/TestMessageSetProcessor.java | 5 +- .../thrift}/TestMessageSetSerDe.java | 3 +- .../queue/TestMessageSetProcessorManager.java | 62 -------- .../suro/routing/TestMessageRouter.java | 4 +- .../server/SuroServerExternalResource.java | 12 +- .../netflix/suro/server/TestStatusServer.java | 5 +- .../netflix/suro/sink/notice/TestNotice.java | 104 ++++++------- 42 files changed, 781 insertions(+), 360 deletions(-) create mode 100644 suro-core/src/main/java/com/netflix/suro/input/SuroInput.java create mode 100644 suro-kafka/src/main/java/com/netflix/suro/input/kafka/KafkaConsumer.java create mode 100644 suro-kafka/src/test/java/com/netflix/suro/input/kafka/TestKafkaConsumer.java create mode 100644 suro-server/conf/input.json create mode 100644 suro-server/src/main/java/com/netflix/suro/input/DynamicPropertyInputConfigurator.java create mode 100644 suro-server/src/main/java/com/netflix/suro/input/InputManager.java create mode 100644 suro-server/src/main/java/com/netflix/suro/input/SuroInputPlugin.java rename suro-server/src/main/java/com/netflix/suro/{server => input/thrift}/CustomServerSocket.java (99%) rename suro-server/src/main/java/com/netflix/suro/{queue => input/thrift}/MessageSetProcessor.java (97%) rename suro-server/src/main/java/com/netflix/suro/{queue => input/thrift}/MessageSetSerDe.java (98%) rename suro-server/src/main/java/com/netflix/suro/{server => input/thrift}/ServerConfig.java (99%) rename suro-server/src/main/java/com/netflix/suro/{server => input/thrift}/ThriftServer.java (69%) delete mode 100644 suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessorManager.java create mode 100644 suro-server/src/test/java/com/netflix/suro/input/TestInputManager.java rename suro-server/src/test/java/com/netflix/suro/{queue => input/thrift}/TestMessageSetProcessor.java (95%) rename suro-server/src/test/java/com/netflix/suro/{queue => input/thrift}/TestMessageSetSerDe.java (95%) delete mode 100644 suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessorManager.java diff --git a/build.gradle b/build.gradle index a00dc935..420d504a 100644 --- a/build.gradle +++ b/build.gradle @@ -118,7 +118,9 @@ project(':suro-server') { compile 'commons-cli:commons-cli:1.2' + testCompile 'org.apache.curator:curator-test:2.4.2' testCompile project(':suro-client').sourceSets.test.output + testCompile project(':suro-kafka').sourceSets.test.output } mainClassName = "com.netflix.suro.SuroServer" @@ -126,7 +128,7 @@ project(':suro-server') { main = "com.netflix.suro.SuroServer" classpath = sourceSets.main.runtimeClasspath classpath += sourceSets.main.compileClasspath - args = ['-m', 'conf/routingmap.json', '-s', 'conf/sink.json'] + args = ['-m', 'conf/routingmap.json', '-s', 'conf/sink.json', '-i', 'conf/input.json'] } installApp { into('build/install/'+applicationName + '-' + version){ diff --git a/suro-core/src/main/java/com/netflix/suro/SuroPlugin.java b/suro-core/src/main/java/com/netflix/suro/SuroPlugin.java index cbca3cc9..d914f081 100644 --- a/suro-core/src/main/java/com/netflix/suro/SuroPlugin.java +++ b/suro-core/src/main/java/com/netflix/suro/SuroPlugin.java @@ -2,6 +2,7 @@ import com.google.inject.AbstractModule; import com.google.inject.multibindings.Multibinder; +import com.netflix.suro.input.SuroInput; import com.netflix.suro.routing.Filter; import com.netflix.suro.sink.Sink; import com.netflix.suro.sink.notice.Notice; @@ -33,6 +34,14 @@ public void addSinkType(String typeName, Class sinkClass) { bindings.addBinding().toInstance(new TypeHolder(typeName, sinkClass)); } + public void addInputType(String typeName, Class inputClass) { + LOG.info("Adding inputType: " + typeName + " -> " + inputClass.getCanonicalName()); + + Multibinder bindings + = Multibinder.newSetBinder(binder(), TypeHolder.class); + bindings.addBinding().toInstance(new TypeHolder(typeName, inputClass)); + } + public void addRemotePrefixFormatterType(String typeName, Class remotePrefixFormatterClass) { LOG.info("Adding remotePrefixFormatterType: " + typeName + " -> " + remotePrefixFormatterClass.getCanonicalName()); diff --git a/suro-core/src/main/java/com/netflix/suro/input/SuroInput.java b/suro-core/src/main/java/com/netflix/suro/input/SuroInput.java new file mode 100644 index 00000000..7982d06a --- /dev/null +++ b/suro-core/src/main/java/com/netflix/suro/input/SuroInput.java @@ -0,0 +1,13 @@ +package com.netflix.suro.input; + +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +public interface SuroInput { + String getId(); + void start() throws Exception; + void shutdown(); + + void startTakingTraffic(); + void stopTakingTraffic(); +} diff --git a/suro-core/src/main/java/com/netflix/suro/jackson/DefaultObjectMapper.java b/suro-core/src/main/java/com/netflix/suro/jackson/DefaultObjectMapper.java index 19f5f7e7..288f5fc1 100644 --- a/suro-core/src/main/java/com/netflix/suro/jackson/DefaultObjectMapper.java +++ b/suro-core/src/main/java/com/netflix/suro/jackson/DefaultObjectMapper.java @@ -84,7 +84,11 @@ public Object findInjectableValue( try { return injector.getInstance(Key.get(forProperty.getType().getRawClass(), Names.named((String)valueId))); } catch (Exception e) { - LOG.info("No implementation found, returning null"); + try { + return injector.getInstance(forProperty.getType().getRawClass()); + } catch (Exception ex) { + LOG.info("No implementation found, returning null"); + } return null; } } diff --git a/suro-core/src/main/java/com/netflix/suro/queue/TrafficController.java b/suro-core/src/main/java/com/netflix/suro/queue/TrafficController.java index 84af9923..905d38a4 100644 --- a/suro-core/src/main/java/com/netflix/suro/queue/TrafficController.java +++ b/suro-core/src/main/java/com/netflix/suro/queue/TrafficController.java @@ -1,9 +1,32 @@ package com.netflix.suro.queue; -public interface TrafficController { - void stopTakingTraffic(); +import com.netflix.governator.guice.lazy.LazySingleton; +import com.netflix.suro.input.SuroInput; - void startTakingTraffic(); +import java.util.ArrayList; +import java.util.List; - int getStatus(); +@LazySingleton +public class TrafficController { + private List inputList = new ArrayList(); + + public void stopTakingTraffic() { + for (SuroInput input : inputList) { + input.stopTakingTraffic(); + } + } + + public void startTakingTraffic() { + for (SuroInput input : inputList) { + input.startTakingTraffic(); + } + } + + public void registerService(SuroInput input) { + inputList.add(input); + } + + public void unregisterService(SuroInput input) { + inputList.remove(input); + } } diff --git a/suro-core/src/main/java/com/netflix/suro/routing/MessageRouter.java b/suro-core/src/main/java/com/netflix/suro/routing/MessageRouter.java index 8cfdf67a..c495822e 100644 --- a/suro-core/src/main/java/com/netflix/suro/routing/MessageRouter.java +++ b/suro-core/src/main/java/com/netflix/suro/routing/MessageRouter.java @@ -17,7 +17,7 @@ package com.netflix.suro.routing; import com.google.inject.Inject; -import com.netflix.governator.guice.lazy.LazySingleton; +import com.google.inject.Singleton; import com.netflix.servo.monitor.Monitors; import com.netflix.suro.message.MessageContainer; import com.netflix.suro.routing.RoutingMap.Route; @@ -32,7 +32,7 @@ * * @author jbae */ -@LazySingleton +@Singleton public class MessageRouter { private static final Logger log = LoggerFactory.getLogger(MessageRouter.class); diff --git a/suro-core/src/main/java/com/netflix/suro/sink/SinkManager.java b/suro-core/src/main/java/com/netflix/suro/sink/SinkManager.java index ec07a6f1..ea7754e3 100644 --- a/suro-core/src/main/java/com/netflix/suro/sink/SinkManager.java +++ b/suro-core/src/main/java/com/netflix/suro/sink/SinkManager.java @@ -18,7 +18,8 @@ import com.google.common.collect.Maps; import com.google.inject.Singleton; -import org.apache.log4j.Logger; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.PreDestroy; import java.util.Collection; @@ -32,7 +33,7 @@ */ @Singleton public class SinkManager { - private static final Logger log = Logger.getLogger(SinkManager.class); + private static final Logger log = LoggerFactory.getLogger(SinkManager.class); private final ConcurrentMap sinkMap = Maps.newConcurrentMap(); diff --git a/suro-integration-test/src/test/java/com/netflix/suro/TestSuroServer.java b/suro-integration-test/src/test/java/com/netflix/suro/TestSuroServer.java index 396993d3..7920b109 100644 --- a/suro-integration-test/src/test/java/com/netflix/suro/TestSuroServer.java +++ b/suro-integration-test/src/test/java/com/netflix/suro/TestSuroServer.java @@ -35,7 +35,11 @@ import static org.junit.Assert.assertTrue; public class TestSuroServer { - private static final Properties properties = new Properties(); + private static final String inputConfig = "[\n" + + " {\n" + + " \"type\": \"thrift\"\n" + + " }\n" + + "]"; private static final String sinkDesc = "{\n" + " \"default\": {\n" + " \"type\": \"TestSink\",\n" + @@ -73,7 +77,7 @@ public class TestSuroServer { "}"; @Rule - public SuroServerExternalResource suroServer = new SuroServerExternalResource(sinkDesc, mapDesc); + public SuroServerExternalResource suroServer = new SuroServerExternalResource(inputConfig, sinkDesc, mapDesc); @Test public void test() throws Exception { diff --git a/suro-kafka/src/main/java/com/netflix/suro/input/kafka/KafkaConsumer.java b/suro-kafka/src/main/java/com/netflix/suro/input/kafka/KafkaConsumer.java new file mode 100644 index 00000000..5073aae2 --- /dev/null +++ b/suro-kafka/src/main/java/com/netflix/suro/input/kafka/KafkaConsumer.java @@ -0,0 +1,142 @@ +package com.netflix.suro.input.kafka; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import com.netflix.suro.input.SuroInput; +import com.netflix.suro.message.DefaultMessageContainer; +import com.netflix.suro.message.Message; +import com.netflix.suro.routing.MessageRouter; +import kafka.consumer.*; +import kafka.javaapi.consumer.ConsumerConnector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +public class KafkaConsumer implements SuroInput { + public static final String TYPE = "kafka"; + private static Logger log = LoggerFactory.getLogger(KafkaConsumer.class); + + private final Properties consumerProps; + private final String topic; + private final MessageRouter router; + private final ObjectMapper jsonMapper; + + private ConsumerConnector connector; + private ConsumerIterator stream; + private ExecutorService executor = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setNameFormat("KafkaConsumer-%d").build()); + private Future runner = null; + private boolean running = false; + + @JsonCreator + public KafkaConsumer( + @JsonProperty("consumerProps") Properties consumerProps, + @JsonProperty("topic") String topic, + @JacksonInject MessageRouter router, + @JacksonInject ObjectMapper jsonMapper + ) { + Preconditions.checkNotNull(consumerProps); + Preconditions.checkNotNull(topic); + Preconditions.checkNotNull(consumerProps.getProperty("group.id")); + Preconditions.checkNotNull(consumerProps.getProperty("zookeeper.connect")); + String timeoutStr = consumerProps.getProperty("consumer.timeout.ms"); + Preconditions.checkNotNull(timeoutStr); + Preconditions.checkArgument(Long.parseLong(timeoutStr) > 0); + + this.consumerProps = consumerProps; + this.topic = topic; + this.router = router; + this.jsonMapper = jsonMapper; + } + + @Override + public String getId() { + return topic + "-" + consumerProps.getProperty("group.id"); + } + + @Override + public void start() throws Exception { + connector = Consumer.createJavaConsumerConnector(new ConsumerConfig(consumerProps)); + + final Map>> streams = connector.createMessageStreams(ImmutableMap.of(topic, 1)); + + final List> streamList = streams.get(topic); + if (streamList == null || streamList.size() != 1) { + throw new RuntimeException(topic + " is not valid"); + } + + stream = streamList.get(0).iterator(); + + startTakingTraffic(); + } + + @Override + public void shutdown() { + stopTakingTraffic(); + connector.shutdown(); + } + + @Override + public void startTakingTraffic() { + running = true; + runner = executor.submit(new Runnable() { + @Override + public void run() { + while (running) { + try { + byte[] message = stream.next().message(); + router.process(new DefaultMessageContainer(new Message(topic, message), jsonMapper)); + } catch (ConsumerTimeoutException timeoutException) { + // do nothing + } catch (Exception e) { + log.error("Exception on consuming kafka with topic: " + topic, e); + } + } + } + }); + } + + @Override + public void stopTakingTraffic() { + running = false; + try { + runner.get(); + } catch (InterruptedException e) { + // do nothing + } catch (ExecutionException e) { + log.error("Exception on stopping the task", e); + } + } + + @Override + public boolean equals(Object o) { + if (o instanceof KafkaConsumer) { + KafkaConsumer kafkaConsumer = (KafkaConsumer) o; + boolean topicEquals = topic.equals(kafkaConsumer.topic); + if (topicEquals) { + return consumerProps.getProperty("group.id").equals(kafkaConsumer.consumerProps.getProperty("group.id")); + } else { + return false; + } + } else { + return false; + } + } + + @Override + public int hashCode() { + return (getId()).hashCode(); + } +} diff --git a/suro-kafka/src/test/java/com/netflix/suro/input/kafka/TestKafkaConsumer.java b/suro-kafka/src/test/java/com/netflix/suro/input/kafka/TestKafkaConsumer.java new file mode 100644 index 00000000..5d111fff --- /dev/null +++ b/suro-kafka/src/test/java/com/netflix/suro/input/kafka/TestKafkaConsumer.java @@ -0,0 +1,138 @@ +package com.netflix.suro.input.kafka; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.netflix.suro.jackson.DefaultObjectMapper; +import com.netflix.suro.message.Message; +import com.netflix.suro.message.MessageContainer; +import com.netflix.suro.message.MessageSetReader; +import com.netflix.suro.message.StringMessage; +import com.netflix.suro.routing.MessageRouter; +import com.netflix.suro.sink.Sink; +import com.netflix.suro.sink.kafka.KafkaServerExternalResource; +import com.netflix.suro.sink.kafka.KafkaSink; +import com.netflix.suro.sink.kafka.TestKafkaSink; +import com.netflix.suro.sink.kafka.ZkExternalResource; +import kafka.admin.TopicCommand; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; +import org.mockito.ArgumentCaptor; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import java.util.Iterator; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +public class TestKafkaConsumer { + public static ZkExternalResource zk = new ZkExternalResource(); + public static KafkaServerExternalResource kafkaServer = new KafkaServerExternalResource(zk); + + @ClassRule + public static TestRule chain = RuleChain + .outerRule(zk) + .around(kafkaServer); + + private static final String TOPIC_NAME = "testkafkaconsumer"; + @Test + public void test() throws Exception { + TopicCommand.createTopic(zk.getZkClient(), + new TopicCommand.TopicCommandOptions(new String[]{ + "--zookeeper", "dummy", "--create", "--topic", TOPIC_NAME, + "--replication-factor", "2", "--partitions", "1"})); + + ObjectMapper jsonMapper = new DefaultObjectMapper(); + + sendKafkaMessage(jsonMapper); + + final CountDownLatch latch = new CountDownLatch(2); + + MessageRouter router = mock(MessageRouter.class); + doAnswer(new Answer() { + + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + latch.countDown(); + return null; + } + }).when(router).process(any(MessageContainer.class)); + + Properties properties = new Properties(); + properties.setProperty("group.id", "testkafkaconsumer"); + properties.setProperty("zookeeper.connect", zk.getConnectionString()); + properties.setProperty("auto.offset.reset", "smallest"); + + KafkaConsumer consumer = null; + try { + consumer = new KafkaConsumer(properties, TOPIC_NAME, router, jsonMapper); + fail("should have failed without timeout"); + } catch (Exception e) { + // do nothing + } + + properties.setProperty("consumer.timeout.ms", "1000"); + consumer = new KafkaConsumer(properties, TOPIC_NAME, router, jsonMapper); + + consumer.start(); + latch.await(500000, TimeUnit.MILLISECONDS); + + ArgumentCaptor msgContainers = ArgumentCaptor.forClass(MessageContainer.class); + verify(router, times(2)).process(msgContainers.capture()); + for (MessageContainer container : msgContainers.getAllValues()) { + assertEquals(container.getRoutingKey(), TOPIC_NAME); + assertTrue(container.getEntity(String.class).startsWith("testMessage")); + } + consumer.stopTakingTraffic(); + + final CountDownLatch latch1 = new CountDownLatch(2); + doAnswer(new Answer() { + + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + latch1.countDown(); + return null; + } + }).when(router).process(any(MessageContainer.class)); + + sendKafkaMessage(jsonMapper); + latch1.await(5000, TimeUnit.MILLISECONDS); + assertEquals(latch1.getCount(), 2); // not processed due to stop + + consumer.startTakingTraffic(); + latch1.await(5000, TimeUnit.MILLISECONDS); + assertEquals(latch1.getCount(), 0); + verify(router, times(4)).process(msgContainers.capture()); + for (MessageContainer container : msgContainers.getAllValues()) { + assertEquals(container.getRoutingKey(), TOPIC_NAME); + assertTrue(container.getEntity(String.class).startsWith("testMessage")); + } + } + + private void sendKafkaMessage(ObjectMapper jsonMapper) throws java.io.IOException, InterruptedException { + String description = "{\n" + + " \"type\": \"kafka\",\n" + + " \"client.id\": \"kafkasink\",\n" + + " \"metadata.broker.list\": \"" + kafkaServer.getBrokerListStr() + "\",\n" + + " \"request.required.acks\": 1\n" + + "}"; + + jsonMapper.registerSubtypes(new NamedType(KafkaSink.class, "kafka")); + KafkaSink sink = jsonMapper.readValue(description, new TypeReference(){}); + sink.open(); + Iterator msgIterator = new MessageSetReader(TestKafkaSink.createMessageSet(TOPIC_NAME, 2)).iterator(); + while (msgIterator.hasNext()) { + sink.writeTo(new StringMessage(msgIterator.next())); + } + sink.close(); + } +} diff --git a/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/TestKafkaSink.java b/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/TestKafkaSink.java index 9981ffdc..8da20f97 100644 --- a/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/TestKafkaSink.java +++ b/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/TestKafkaSink.java @@ -69,7 +69,7 @@ public void testDefaultParameters() throws IOException { jsonMapper.registerSubtypes(new NamedType(KafkaSink.class, "kafka")); KafkaSink sink = jsonMapper.readValue(description, new TypeReference(){}); sink.open(); - Iterator msgIterator = new MessageSetReader(createMessageSet(2)).iterator(); + Iterator msgIterator = new MessageSetReader(createMessageSet(TOPIC_NAME, 2)).iterator(); while (msgIterator.hasNext()) { sink.writeTo(new StringMessage(msgIterator.next())); } @@ -253,10 +253,10 @@ private byte[] extractMessage(List messageSet, int offset) { return bytes; } - public static TMessageSet createMessageSet(int numMsgs) { + public static TMessageSet createMessageSet(String topic, int numMsgs) { MessageSetBuilder builder = new MessageSetBuilder(new ClientConfig()).withCompression(Compression.LZF); for (int i = 0; i < numMsgs; ++i) { - builder.withMessage(TOPIC_NAME, ("testMessage" + i).getBytes()); + builder.withMessage(topic, ("testMessage" + i).getBytes()); } return builder.build(); diff --git a/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/ZkExternalResource.java b/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/ZkExternalResource.java index 02372ea8..2c1c7c2e 100644 --- a/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/ZkExternalResource.java +++ b/suro-kafka/src/test/java/com/netflix/suro/sink/kafka/ZkExternalResource.java @@ -61,4 +61,5 @@ public ZkClient getZkClient() { return zkClient; } public int getServerPort() { return zkServer.getPort(); } + public String getConnectionString() { return zkServer.getConnectString(); } } diff --git a/suro-localfile/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java b/suro-localfile/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java index 9119496c..df27f4cc 100644 --- a/suro-localfile/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java +++ b/suro-localfile/src/main/java/com/netflix/suro/sink/localfile/LocalFileSink.java @@ -97,8 +97,8 @@ public LocalFileSink( @JsonProperty("queue4Sink") MessageQueue4Sink queue4Sink, @JsonProperty("batchSize") int batchSize, @JsonProperty("batchTimeout") int batchTimeout, - @JacksonInject("queueManager") TrafficController trafficController, - @JacksonInject("spaceChecker") SpaceChecker spaceChecker) { + @JacksonInject TrafficController trafficController, + @JacksonInject SpaceChecker spaceChecker) { if (!outputDir.endsWith("/")) { outputDir += "/"; } @@ -128,22 +128,7 @@ public void open() { spaceChecker = new SpaceChecker(minPercentFreeDisk, outputDir); } if (trafficController == null) { - trafficController = new TrafficController() { - @Override - public void stopTakingTraffic() { - - } - - @Override - public void startTakingTraffic() { - - } - - @Override - public int getStatus() { - return 0; - } - }; + trafficController = new TrafficController(); } notice.init(); diff --git a/suro-localfile/src/test/java/com/netflix/suro/sink/localfile/TestLocalFileSink.java b/suro-localfile/src/test/java/com/netflix/suro/sink/localfile/TestLocalFileSink.java index 3e7df89d..ebf48153 100644 --- a/suro-localfile/src/test/java/com/netflix/suro/sink/localfile/TestLocalFileSink.java +++ b/suro-localfile/src/test/java/com/netflix/suro/sink/localfile/TestLocalFileSink.java @@ -17,9 +17,6 @@ package com.netflix.suro.sink.localfile; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.BeanProperty; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.AbstractModule; import com.google.inject.Guice; @@ -35,17 +32,11 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; +import java.io.*; import java.util.HashSet; import java.util.Set; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.*; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -65,11 +56,6 @@ public void stopTakingTraffic() { public void startTakingTraffic() { status = 200; } - - @Override - public int getStatus() { - return status; - } }; private static Injector injector = Guice.createInjector( @@ -93,16 +79,6 @@ public void testDefaultParameters() throws IOException { "}"; ObjectMapper mapper = injector.getInstance(ObjectMapper.class); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else { - return null; - } - } - }); Sink sink = mapper.readValue(localFileSinkSpec, new TypeReference(){}); sink.open(); @@ -153,16 +129,6 @@ public void testWithPeriodRotation() throws IOException, InterruptedException { "}"; ObjectMapper mapper = injector.getInstance(ObjectMapper.class); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else { - return null; - } - } - }); Sink sink = mapper.readValue(localFileSinkSpec, new TypeReference(){}); sink.open(); assertNull(sink.recvNotice()); @@ -218,26 +184,11 @@ public void testSpaceChecker() throws Exception { final LocalFileSink.SpaceChecker spaceChecker = mock(LocalFileSink.SpaceChecker.class); when(spaceChecker.hasEnoughSpace()).thenReturn(false); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else if (valueId.equals("spaceChecker")) { - return spaceChecker; - } else { - return null; - } - } - }); - assertEquals(trafficController.getStatus(), 200); - Sink sink = mapper.readValue(localFileSinkSpec, new TypeReference(){}); sink.open(); Thread.sleep(1000); // wait until thread starts - assertEquals(trafficController.getStatus(), 503); assertNull(sink.recvNotice()); when(spaceChecker.hasEnoughSpace()).thenReturn(true); @@ -286,17 +237,6 @@ public void testWithSizeRotation() throws IOException { "}"; ObjectMapper mapper = injector.getInstance(ObjectMapper.class); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else { - return null; - } - - } - }); Sink sink = mapper.readValue(localFileSinkSpec, new TypeReference(){}); sink.open(); assertNull(sink.recvNotice()); @@ -351,16 +291,6 @@ public void rotateEmptyFile() throws IOException, InterruptedException { "}"; ObjectMapper mapper = injector.getInstance(ObjectMapper.class); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else { - return null; - } - } - }); Sink sink = mapper.readValue(localFileSinkSpec, new TypeReference(){}); sink.open(); assertNull(sink.recvNotice()); @@ -424,16 +354,6 @@ public void testCleanUp() throws IOException, InterruptedException { Thread.sleep(3000); // wait until .suro file is expired ObjectMapper mapper = injector.getInstance(ObjectMapper.class); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue(Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance) { - if (valueId.equals("queueManager")) { - return trafficController; - } else { - return null; - } - } - }); LocalFileSink sink = (LocalFileSink)mapper.readValue( localFileSinkSpec, new TypeReference(){}); diff --git a/suro-s3/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java b/suro-s3/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java index 013785d4..0fdd77ff 100644 --- a/suro-s3/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java +++ b/suro-s3/src/main/java/com/netflix/suro/sink/remotefile/S3FileSink.java @@ -85,8 +85,8 @@ public S3FileSink( @JsonProperty("batchUpload") boolean batchUpload, @JsonProperty("s3Acl") String s3Acl, @JsonProperty("s3AclRetries") int s3AclRetries, - @JacksonInject("multipartUtils") MultipartUtils mpUtils, - @JacksonInject("credentials") AWSCredentialsProvider credentialProvider) { + @JacksonInject MultipartUtils mpUtils, + @JacksonInject AWSCredentialsProvider credentialProvider) { super(localFileSink, prefixFormatter, concurrentUpload, batchUpload); this.bucket = bucket; diff --git a/suro-s3/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java b/suro-s3/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java index c73868e5..32a3d1d1 100644 --- a/suro-s3/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java +++ b/suro-s3/src/test/java/com/netflix/suro/sink/remotefile/TestS3FileSink.java @@ -23,7 +23,6 @@ import com.google.inject.AbstractModule; import com.google.inject.Guice; import com.google.inject.Injector; -import com.google.inject.name.Names; import com.netflix.suro.connection.TestConnectionPool; import com.netflix.suro.jackson.DefaultObjectMapper; import com.netflix.suro.message.Message; @@ -54,9 +53,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.*; public class TestS3FileSink { @Rule @@ -294,7 +291,6 @@ private Injector getInjector() { protected void configure() { bind(ObjectMapper.class).to(DefaultObjectMapper.class); bind(AWSCredentialsProvider.class) - .annotatedWith(Names.named("credentials")) .toInstance(new AWSCredentialsProvider() { @Override public AWSCredentials getCredentials() { @@ -330,18 +326,12 @@ public Object answer(InvocationOnMock invocation) throws Throwable { any(List.class), any(S3ServiceEventListener.class)); - bind(MultipartUtils.class) - .annotatedWith(Names.named("multipartUtils")) - .toInstance(mpUtils); + bind(MultipartUtils.class).toInstance(mpUtils); } catch (Exception e) { Assert.fail(e.getMessage()); } - bind(TrafficController.class) - .annotatedWith(Names.named("queueManager")) - .toInstance(mock(TrafficController.class)); - bind(SpaceChecker.class) - .annotatedWith(Names.named("spaceChecker")) - .toInstance(mock(SpaceChecker.class)); + bind(TrafficController.class).toInstance(mock(TrafficController.class)); + bind(SpaceChecker.class).toInstance(mock(SpaceChecker.class)); } } ); diff --git a/suro-server/conf/input.json b/suro-server/conf/input.json new file mode 100644 index 00000000..038910f0 --- /dev/null +++ b/suro-server/conf/input.json @@ -0,0 +1,5 @@ +[ + { + "type": "thrift" + } +] \ No newline at end of file diff --git a/suro-server/src/main/java/com/netflix/suro/SuroDynamicPropertyModule.java b/suro-server/src/main/java/com/netflix/suro/SuroDynamicPropertyModule.java index 157c85bc..fbad345c 100644 --- a/suro-server/src/main/java/com/netflix/suro/SuroDynamicPropertyModule.java +++ b/suro-server/src/main/java/com/netflix/suro/SuroDynamicPropertyModule.java @@ -17,6 +17,7 @@ package com.netflix.suro; import com.google.inject.AbstractModule; +import com.netflix.suro.input.DynamicPropertyInputConfigurator; import com.netflix.suro.routing.DynamicPropertyRoutingMapConfigurator; import com.netflix.suro.sink.DynamicPropertySinkConfigurator; @@ -28,6 +29,7 @@ public class SuroDynamicPropertyModule extends AbstractModule { @Override protected void configure() { + bind(DynamicPropertyInputConfigurator.class).asEagerSingleton(); bind(DynamicPropertySinkConfigurator.class).asEagerSingleton(); bind(DynamicPropertyRoutingMapConfigurator.class).asEagerSingleton(); } diff --git a/suro-server/src/main/java/com/netflix/suro/SuroModule.java b/suro-server/src/main/java/com/netflix/suro/SuroModule.java index 1d1ab152..727b8a2f 100644 --- a/suro-server/src/main/java/com/netflix/suro/SuroModule.java +++ b/suro-server/src/main/java/com/netflix/suro/SuroModule.java @@ -19,15 +19,14 @@ import com.amazonaws.auth.AWSCredentialsProvider; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.AbstractModule; -import com.google.inject.name.Names; import com.netflix.suro.aws.PropertyAWSCredentialsProvider; +import com.netflix.suro.input.thrift.MessageSetProcessor; +import com.netflix.suro.input.thrift.ServerConfig; import com.netflix.suro.jackson.DefaultObjectMapper; import com.netflix.suro.routing.RoutingMap; import com.netflix.suro.server.StatusServer; import com.netflix.suro.sink.SinkManager; -import java.util.Properties; - /** * Guice module for binding {@link AWSCredentialsProvider}, * Jackson {@link ObjectMapper}, {@link SinkManager}, {@link RoutingMap}, @@ -38,12 +37,11 @@ public class SuroModule extends AbstractModule { @Override protected void configure() { - bind(AWSCredentialsProvider.class) - .annotatedWith(Names.named("credentials")).to(PropertyAWSCredentialsProvider.class); - bind(ObjectMapper.class).to(DefaultObjectMapper.class).asEagerSingleton(); bind(AWSCredentialsProvider.class).to(PropertyAWSCredentialsProvider.class); - bind(SuroService.class); - bind(StatusServer.class); + bind(SuroService.class).asEagerSingleton(); + bind(StatusServer.class).asEagerSingleton(); + bind(ServerConfig.class).asEagerSingleton(); + bind(MessageSetProcessor.class).asEagerSingleton(); } } diff --git a/suro-server/src/main/java/com/netflix/suro/SuroServer.java b/suro-server/src/main/java/com/netflix/suro/SuroServer.java index c3dcc9f4..1f25e646 100644 --- a/suro-server/src/main/java/com/netflix/suro/SuroServer.java +++ b/suro-server/src/main/java/com/netflix/suro/SuroServer.java @@ -24,6 +24,8 @@ import com.netflix.governator.guice.BootstrapModule; import com.netflix.governator.guice.LifecycleInjector; import com.netflix.governator.lifecycle.LifecycleManager; +import com.netflix.suro.input.DynamicPropertyInputConfigurator; +import com.netflix.suro.input.SuroInputPlugin; import com.netflix.suro.routing.DynamicPropertyRoutingMapConfigurator; import com.netflix.suro.routing.RoutingPlugin; import com.netflix.suro.server.StatusServer; @@ -74,6 +76,9 @@ public static void main(String[] args) throws IOException { } else if (propName.equals(DynamicPropertySinkConfigurator.SINK_PROPERTY)) { properties.setProperty(DynamicPropertySinkConfigurator.SINK_PROPERTY, FileUtils.readFileToString(new File(value))); + } else if (propName.equals(DynamicPropertyInputConfigurator.INPUT_CONFIG_PROPERTY)) { + properties.setProperty(DynamicPropertyInputConfigurator.INPUT_CONFIG_PROPERTY, + FileUtils.readFileToString(new File(value))); } else { properties.setProperty(propName, value); } @@ -117,6 +122,7 @@ public void configure(BootstrapBinder binder) { .withModules( new RoutingPlugin(), new ServerSinkPlugin(), + new SuroInputPlugin(), new SuroDynamicPropertyModule(), new SuroModule(), StatusServer.createJerseyServletModule() @@ -158,6 +164,12 @@ private static Options createOptions() { .withDescription("sink") .create('s'); + Option inputFile = OptionBuilder.withArgName("inputConfig" ) + .hasArg() + .isRequired(true) + .withDescription("input") + .create('i'); + Option accessKey = OptionBuilder.withArgName("AWSAccessKey" ) .hasArg() .isRequired(false) @@ -180,6 +192,7 @@ private static Options createOptions() { options.addOption(propertyFile); options.addOption(mapFile); options.addOption(sinkFile); + options.addOption(inputFile); options.addOption(accessKey); options.addOption(secretKey); options.addOption(controlPort); diff --git a/suro-server/src/main/java/com/netflix/suro/SuroService.java b/suro-server/src/main/java/com/netflix/suro/SuroService.java index a4cebe93..5a4bc645 100644 --- a/suro-server/src/main/java/com/netflix/suro/SuroService.java +++ b/suro-server/src/main/java/com/netflix/suro/SuroService.java @@ -19,9 +19,10 @@ import com.google.common.base.Throwables; import com.google.inject.Inject; import com.google.inject.Singleton; -import com.netflix.suro.queue.MessageSetProcessor; +import com.netflix.suro.input.InputManager; +import com.netflix.suro.input.thrift.MessageSetProcessor; import com.netflix.suro.server.StatusServer; -import com.netflix.suro.server.ThriftServer; +import com.netflix.suro.input.thrift.ThriftServer; import com.netflix.suro.sink.SinkManager; import org.apache.log4j.Logger; @@ -40,25 +41,21 @@ public class SuroService { static Logger log = Logger.getLogger(SuroServer.class); private final StatusServer statusServer; - private final ThriftServer server; - private final MessageSetProcessor queue; + private final InputManager inputManager; private final SinkManager sinkManager; @Inject - private SuroService(StatusServer statusServer, ThriftServer thriftServer, MessageSetProcessor queue, SinkManager sinkManager) { + private SuroService(StatusServer statusServer, InputManager inputManager, SinkManager sinkManager) { this.statusServer = statusServer; - this.server = thriftServer; - this.queue = queue; + this.inputManager = inputManager; this.sinkManager = sinkManager; } @PostConstruct public void start() { try { - queue.start(); - server.start(); statusServer.start(); - } + } catch (Exception e) { log.error("Exception while starting up server: " + e.getMessage(), e); Throwables.propagate(e); @@ -68,9 +65,8 @@ public void start() { @PreDestroy public void shutdown() { try { - server .shutdown(); + inputManager.shutdown(); statusServer.shutdown(); - queue .shutdown(); sinkManager .shutdown(); } catch (Exception e) { //ignore every exception while shutting down but loggign should be done for debugging diff --git a/suro-server/src/main/java/com/netflix/suro/input/DynamicPropertyInputConfigurator.java b/suro-server/src/main/java/com/netflix/suro/input/DynamicPropertyInputConfigurator.java new file mode 100644 index 00000000..c54a9d5d --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/input/DynamicPropertyInputConfigurator.java @@ -0,0 +1,55 @@ +package com.netflix.suro.input; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import com.netflix.config.DynamicStringProperty; +import com.netflix.governator.annotations.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.PostConstruct; +import java.util.List; + +public class DynamicPropertyInputConfigurator { + public static final String INPUT_CONFIG_PROPERTY = "SuroServer.inputConfig"; + + private static Logger LOG = LoggerFactory.getLogger(DynamicPropertyInputConfigurator.class); + + private final InputManager inputManager; + private final ObjectMapper jsonMapper; + + @Configuration(INPUT_CONFIG_PROPERTY) + private String initialInputConfig; + + @Inject + public DynamicPropertyInputConfigurator( + InputManager inputManager, + ObjectMapper jsonMapper) { + this.inputManager = inputManager; + this.jsonMapper = jsonMapper; + } + + @PostConstruct + public void init() { + DynamicStringProperty routingMapFP = new DynamicStringProperty(INPUT_CONFIG_PROPERTY, initialInputConfig) { + @Override + protected void propertyChanged() { + buildInput(get()); + } + }; + + buildInput(routingMapFP.get()); + } + + private void buildInput(String inputListStr) { + try { + List inputList = jsonMapper.readValue( + inputListStr, + new TypeReference>() {}); + inputManager.set(inputList); + } catch (Exception e) { + LOG.info("Error reading input config from fast property: "+e.getMessage(), e); + } + } +} diff --git a/suro-server/src/main/java/com/netflix/suro/input/InputManager.java b/suro-server/src/main/java/com/netflix/suro/input/InputManager.java new file mode 100644 index 00000000..c351fcb2 --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/input/InputManager.java @@ -0,0 +1,52 @@ +package com.netflix.suro.input; + +import com.google.common.collect.Sets; +import com.google.inject.Singleton; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +@Singleton +public class InputManager { + private static final Logger log = LoggerFactory.getLogger(InputManager.class); + + private ConcurrentMap inputMap = new ConcurrentHashMap(); + + public void set(List inputList) { + for (SuroInput suroInput : inputList) { + if (!inputMap.containsKey(suroInput.getId())) { + try { + suroInput.start(); + inputMap.put(suroInput.getId(), suroInput); + } catch (Exception e) { + log.error("Exception on starting the input", e); + } + } + } + + HashSet suroInputIdSet = Sets.newHashSet(inputList); + + for (Map.Entry e : inputMap.entrySet()) { + if (!suroInputIdSet.contains(e.getValue())) { + inputMap.remove(e.getKey()); + } + } + } + + public SuroInput getInput(String id) { + return inputMap.get(id); + } + + public void shutdown() { + for (SuroInput input : inputMap.values()) { + input.shutdown(); + } + } + + +} diff --git a/suro-server/src/main/java/com/netflix/suro/input/SuroInputPlugin.java b/suro-server/src/main/java/com/netflix/suro/input/SuroInputPlugin.java new file mode 100644 index 00000000..37300ee8 --- /dev/null +++ b/suro-server/src/main/java/com/netflix/suro/input/SuroInputPlugin.java @@ -0,0 +1,13 @@ +package com.netflix.suro.input; + +import com.netflix.suro.SuroPlugin; +import com.netflix.suro.input.kafka.KafkaConsumer; +import com.netflix.suro.input.thrift.ThriftServer; + +public class SuroInputPlugin extends SuroPlugin { + @Override + protected void configure() { + this.addInputType(ThriftServer.TYPE, ThriftServer.class); + this.addInputType(KafkaConsumer.TYPE, KafkaConsumer.class); + } +} diff --git a/suro-server/src/main/java/com/netflix/suro/server/CustomServerSocket.java b/suro-server/src/main/java/com/netflix/suro/input/thrift/CustomServerSocket.java similarity index 99% rename from suro-server/src/main/java/com/netflix/suro/server/CustomServerSocket.java rename to suro-server/src/main/java/com/netflix/suro/input/thrift/CustomServerSocket.java index bd19ddcd..608b03e4 100644 --- a/suro-server/src/main/java/com/netflix/suro/server/CustomServerSocket.java +++ b/suro-server/src/main/java/com/netflix/suro/input/thrift/CustomServerSocket.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.suro.server; +package com.netflix.suro.input.thrift; import org.apache.thrift.transport.TNonblockingServerTransport; import org.apache.thrift.transport.TNonblockingSocket; diff --git a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessor.java b/suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetProcessor.java similarity index 97% rename from suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessor.java rename to suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetProcessor.java index 2013be36..a42f34e5 100644 --- a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessor.java +++ b/suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetProcessor.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.suro.queue; +package com.netflix.suro.input.thrift; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; @@ -30,8 +30,8 @@ import com.netflix.suro.message.Message; import com.netflix.suro.message.MessageSetBuilder; import com.netflix.suro.message.MessageSetReader; +import com.netflix.suro.queue.Queue4Server; import com.netflix.suro.routing.MessageRouter; -import com.netflix.suro.server.ServerConfig; import com.netflix.suro.thrift.*; import org.apache.thrift.TException; import org.slf4j.Logger; @@ -42,7 +42,7 @@ import java.util.concurrent.TimeUnit; /** - * The {@link TMessageSet} processor used by {@link com.netflix.suro.server.ThriftServer}. It takes incoming {@link TMessageSet} + * The {@link TMessageSet} processor used by {@link com.netflix.suro.input.thrift.ThriftServer}. It takes incoming {@link TMessageSet} * sent by Suro client, validates each message set's CRC32 code, and then hands off validated message set to an internal queue. * A {@link MessageRouter} instance will asynchronously route the messages in the queue into configured sinks based on routing rules, * represented by {@link com.amazonaws.services.s3.model.RoutingRule}. @@ -86,7 +86,6 @@ public ServiceStatus getStatus() { public MessageSetProcessor( Queue4Server queue, MessageRouter router, - MessageSetProcessorManager manager, ServerConfig config, ObjectMapper jsonMapper) throws Exception { this.queue = queue; @@ -96,7 +95,6 @@ public MessageSetProcessor( isRunning = true; - manager.registerService(this); Monitors.registerObject(this); } diff --git a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetSerDe.java b/suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetSerDe.java similarity index 98% rename from suro-server/src/main/java/com/netflix/suro/queue/MessageSetSerDe.java rename to suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetSerDe.java index 7b4ec774..06ce687d 100644 --- a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetSerDe.java +++ b/suro-server/src/main/java/com/netflix/suro/input/thrift/MessageSetSerDe.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.suro.queue; +package com.netflix.suro.input.thrift; import com.netflix.suro.message.SerDe; import com.netflix.suro.thrift.TMessageSet; diff --git a/suro-server/src/main/java/com/netflix/suro/server/ServerConfig.java b/suro-server/src/main/java/com/netflix/suro/input/thrift/ServerConfig.java similarity index 99% rename from suro-server/src/main/java/com/netflix/suro/server/ServerConfig.java rename to suro-server/src/main/java/com/netflix/suro/input/thrift/ServerConfig.java index 28757f4b..443e9c32 100644 --- a/suro-server/src/main/java/com/netflix/suro/server/ServerConfig.java +++ b/suro-server/src/main/java/com/netflix/suro/input/thrift/ServerConfig.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.suro.server; +package com.netflix.suro.input.thrift; import com.netflix.governator.annotations.Configuration; diff --git a/suro-server/src/main/java/com/netflix/suro/server/ThriftServer.java b/suro-server/src/main/java/com/netflix/suro/input/thrift/ThriftServer.java similarity index 69% rename from suro-server/src/main/java/com/netflix/suro/server/ThriftServer.java rename to suro-server/src/main/java/com/netflix/suro/input/thrift/ThriftServer.java index 4e4e1870..a3cd049a 100644 --- a/suro-server/src/main/java/com/netflix/suro/server/ThriftServer.java +++ b/suro-server/src/main/java/com/netflix/suro/input/thrift/ThriftServer.java @@ -14,12 +14,14 @@ * limitations under the License. */ -package com.netflix.suro.server; +package com.netflix.suro.input.thrift; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Throwables; -import com.google.inject.Inject; import com.netflix.governator.guice.lazy.LazySingleton; -import com.netflix.suro.queue.MessageSetProcessor; +import com.netflix.suro.input.SuroInput; +import com.netflix.suro.queue.TrafficController; import com.netflix.suro.thrift.SuroServer; import org.apache.thrift.server.THsHaServer; import org.apache.thrift.transport.TTransportException; @@ -29,27 +31,39 @@ import java.util.concurrent.*; @LazySingleton -public class ThriftServer { +public class ThriftServer implements SuroInput { + public static final String TYPE = "thrift"; + private static final Logger logger = LoggerFactory.getLogger(ThriftServer.class); private THsHaServer server = null; private final ServerConfig config; - private final MessageSetProcessor MessageSetProcessor; + private final MessageSetProcessor msgProcessor; private ExecutorService executor; - @Inject + @JsonCreator public ThriftServer( - ServerConfig config, - MessageSetProcessor MessageSetProcessor) throws Exception { + @JacksonInject ServerConfig config, + @JacksonInject MessageSetProcessor msgProcessor, + @JacksonInject TrafficController trafficController) throws Exception { this.config = config; - this.MessageSetProcessor = MessageSetProcessor; + this.msgProcessor = msgProcessor; + trafficController.registerService(this); + } + + @Override + public String getId() { + return TYPE; } + @Override public void start() throws TTransportException { + msgProcessor.start(); + logger.info("Starting ThriftServer with config " + config); CustomServerSocket transport = new CustomServerSocket(config); - SuroServer.Processor processor = new SuroServer.Processor(MessageSetProcessor); + SuroServer.Processor processor = new SuroServer.Processor(msgProcessor); THsHaServer.Args serverArgs = new THsHaServer.Args(transport); serverArgs.workerThreads(config.getThriftWorkerThreadNum()); @@ -87,22 +101,42 @@ public void run() { } } - public boolean isServing(){ - return server != null && server.isServing(); - } - - public boolean isStopped(){ - return server == null || server.isStopped(); - } - + @Override public void shutdown() { logger.info("Shutting down thrift server"); try { + stopTakingTraffic(); + Thread.sleep(1000); server.stop(); executor.shutdownNow(); + msgProcessor.shutdown(); } catch (Exception e) { // ignore any exception when shutdown logger.error("Exception while shutting down: " + e.getMessage(), e); } } + + @Override + public void startTakingTraffic() { + msgProcessor.startTakingTraffic(); + } + + @Override + public void stopTakingTraffic() { + msgProcessor.startTakingTraffic(); + } + + @Override + public boolean equals(Object o) { + if (o instanceof ThriftServer) { + return true; // thrift server is singleton + } else { + return false; + } + } + + @Override + public int hashCode() { + return TYPE.hashCode(); + } } \ No newline at end of file diff --git a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessorManager.java b/suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessorManager.java deleted file mode 100644 index 54aaa28c..00000000 --- a/suro-server/src/main/java/com/netflix/suro/queue/MessageSetProcessorManager.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright 2013 Netflix, 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.netflix.suro.queue; - -import com.google.inject.Singleton; - -@Singleton -public class MessageSetProcessorManager implements TrafficController { - public static final int IN_ERROR = 503; - public static final int OK = 200; - - private volatile int status = OK; - - private MessageSetProcessor service; - public void registerService(MessageSetProcessor service){ - this.service = service; - } - - @Override - public void stopTakingTraffic(){ - if (this.service != null){ - this.service.stopTakingTraffic(); - status = IN_ERROR; - } - } - - @Override - public void startTakingTraffic(){ - if (this.service != null){ - this.service.startTakingTraffic(); - status = OK; - } - } - - @Override - public int getStatus(){ - return status; - } -} diff --git a/suro-server/src/main/java/com/netflix/suro/queue/Queue4Server.java b/suro-server/src/main/java/com/netflix/suro/queue/Queue4Server.java index 7e76c106..d310ab8b 100644 --- a/suro-server/src/main/java/com/netflix/suro/queue/Queue4Server.java +++ b/suro-server/src/main/java/com/netflix/suro/queue/Queue4Server.java @@ -17,7 +17,8 @@ package com.netflix.suro.queue; import com.google.inject.Inject; -import com.netflix.suro.server.ServerConfig; +import com.netflix.suro.input.thrift.MessageSetSerDe; +import com.netflix.suro.input.thrift.ServerConfig; import com.netflix.suro.thrift.TMessageSet; import org.joda.time.Period; import org.slf4j.Logger; @@ -30,7 +31,7 @@ /** * {@link BlockingQueue} wrapper that decides whether delegate queue operations to an in-memory bounded queue, or - * to a disk-backed queue. An in-memory bounded blocking queue will be used if the configuration {@link com.netflix.suro.server.ServerConfig#getQueueType()} + * to a disk-backed queue. An in-memory bounded blocking queue will be used if the configuration {@link com.netflix.suro.input.thrift.ServerConfig#getQueueType()} * returns "memory". Otherwise, a disk-backed queue will be used. * * @author jbae diff --git a/suro-server/src/main/java/com/netflix/suro/server/HealthCheck.java b/suro-server/src/main/java/com/netflix/suro/server/HealthCheck.java index e7d9f649..7768bb32 100644 --- a/suro-server/src/main/java/com/netflix/suro/server/HealthCheck.java +++ b/suro-server/src/main/java/com/netflix/suro/server/HealthCheck.java @@ -18,6 +18,7 @@ import com.google.inject.Inject; import com.google.inject.Singleton; +import com.netflix.suro.input.thrift.ServerConfig; import com.netflix.suro.thrift.ServiceStatus; import com.netflix.suro.thrift.SuroServer; import org.apache.thrift.protocol.TBinaryProtocol; diff --git a/suro-server/src/main/java/com/netflix/suro/server/StatusServer.java b/suro-server/src/main/java/com/netflix/suro/server/StatusServer.java index 39ab74f8..35703a4a 100644 --- a/suro-server/src/main/java/com/netflix/suro/server/StatusServer.java +++ b/suro-server/src/main/java/com/netflix/suro/server/StatusServer.java @@ -22,6 +22,7 @@ import com.google.inject.servlet.GuiceFilter; import com.google.inject.servlet.GuiceServletContextListener; import com.google.inject.servlet.ServletModule; +import com.netflix.suro.input.thrift.ServerConfig; import com.sun.jersey.guice.spi.container.servlet.GuiceContainer; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.DefaultServlet; @@ -55,7 +56,7 @@ protected void configureServlets() { } private final ExecutorService executor = Executors.newSingleThreadExecutor(); - private final ServerConfig config; + private final ServerConfig config; private final Injector injector; private final CountDownLatch startLatch = new CountDownLatch(1); diff --git a/suro-server/src/main/java/com/netflix/suro/sink/notice/SQSNotice.java b/suro-server/src/main/java/com/netflix/suro/sink/notice/SQSNotice.java index 64ed36ad..6d75366f 100644 --- a/suro-server/src/main/java/com/netflix/suro/sink/notice/SQSNotice.java +++ b/suro-server/src/main/java/com/netflix/suro/sink/notice/SQSNotice.java @@ -74,8 +74,8 @@ public SQSNotice( @JsonProperty("socketTimeout") int socketTimeout, @JsonProperty("maxRetries") int maxRetries, @JsonProperty("enableBase64Encoding") boolean enableBase64Encoding, - @JacksonInject("sqsClient") AmazonSQSClient sqsClient, - @JacksonInject("credentials") AWSCredentialsProvider credentialsProvider) { + @JacksonInject AmazonSQSClient sqsClient, + @JacksonInject AWSCredentialsProvider credentialsProvider) { this.queues = queues; this.region = region; diff --git a/suro-server/src/test/java/com/netflix/suro/input/TestInputManager.java b/suro-server/src/test/java/com/netflix/suro/input/TestInputManager.java new file mode 100644 index 00000000..1142bd1d --- /dev/null +++ b/suro-server/src/test/java/com/netflix/suro/input/TestInputManager.java @@ -0,0 +1,128 @@ +package com.netflix.suro.input; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.AbstractModule; +import com.google.inject.Injector; +import com.netflix.governator.configuration.PropertiesConfigurationProvider; +import com.netflix.governator.guice.BootstrapBinder; +import com.netflix.governator.guice.BootstrapModule; +import com.netflix.governator.guice.LifecycleInjector; +import com.netflix.governator.lifecycle.LifecycleManager; +import com.netflix.suro.TestUtils; +import com.netflix.suro.jackson.DefaultObjectMapper; +import com.netflix.suro.sink.kafka.KafkaServerExternalResource; +import com.netflix.suro.sink.kafka.ZkExternalResource; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; + +import java.util.List; +import java.util.Properties; + +import static org.junit.Assert.assertNotNull; + +public class TestInputManager { + public static ZkExternalResource zk = new ZkExternalResource(); + public static KafkaServerExternalResource kafkaServer = new KafkaServerExternalResource(zk); + + @ClassRule + public static TestRule chain = RuleChain + .outerRule(zk) + .around(kafkaServer); + + private String inputConfig = "[\n" + + " {\n" + + " \"type\": \"thrift\"\n" + + " },\n" + + " {\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"kafka_topic\",\n" + + " \"consumerProps\": {\n" + + " \"group.id\": \"kafka1\",\n" + + " \"zookeeper.connect\":\"" + zk.getConnectionString() + "\",\n" + + " \"consumer.timeout.ms\": \"1000\"\n" + + " }\n" + + " }\n" + + "]"; + + private String addInputConfig = "[\n" + + " {\n" + + " \"type\": \"thrift\"\n" + + " },\n" + + " {\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"kafka_topic\",\n" + + " \"consumerProps\": {\n" + + " \"group.id\": \"kafka1\",\n" + + " \"zookeeper.connect\":\"" + zk.getConnectionString() + "\",\n" + + " \"consumer.timeout.ms\": \"1000\"\n" + + " }\n" + + " },\n" + + " {\n" + + " \"type\": \"kafka\",\n" + + " \"topic\": \"kafka_topic\",\n" + + " \"consumerProps\": {\n" + + " \"group.id\": \"kafka2\",\n" + + " \"zookeeper.connect\":\"" + zk.getConnectionString() + "\",\n" + + " \"consumer.timeout.ms\": \"1000\"\n" + + " }\n" + + " }\n" + + "]"; + + @Test + public void test() throws Exception { + int statusPort = TestUtils.pickPort(); + int serverPort = TestUtils.pickPort(); + + final Properties props = new Properties(); + props.put("SuroServer.statusServerPort", Integer.toString(statusPort)); + props.put("SuroServer.port", Integer.toString(serverPort)); + + Injector injector = LifecycleInjector.builder().withBootstrapModule(new BootstrapModule() { + @Override + public void configure(BootstrapBinder binder) { + binder.bindConfigurationProvider().toInstance(new PropertiesConfigurationProvider(props)); + } + }).withModules( + new SuroInputPlugin(), + new AbstractModule() { + @Override + protected void configure() { + bind(ObjectMapper.class).to(DefaultObjectMapper.class); + } + } + ).build().createInjector(); + + LifecycleManager lifecycleManager = injector.getInstance(LifecycleManager.class); + lifecycleManager.start(); + + InputManager inputManager = new InputManager(); + List inputList = injector.getInstance(ObjectMapper.class).readValue( + inputConfig, + new TypeReference>() { + }); + inputManager.set(inputList); + assertNotNull(inputManager.getInput("thrift")); + assertNotNull(inputManager.getInput("kafka_topic-kafka1")); + + inputList = injector.getInstance(ObjectMapper.class).readValue( + addInputConfig, + new TypeReference>() { + }); + inputManager.set(inputList); + assertNotNull(inputManager.getInput("thrift")); + assertNotNull(inputManager.getInput("kafka_topic-kafka1")); + assertNotNull(inputManager.getInput("kafka_topic-kafka2")); + + + inputList = injector.getInstance(ObjectMapper.class).readValue( + inputConfig, + new TypeReference>() { + }); + inputManager.set(inputList); + assertNotNull(inputManager.getInput("thrift")); + assertNotNull(inputManager.getInput("kafka_topic-kafka1")); + } +} diff --git a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessor.java b/suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetProcessor.java similarity index 95% rename from suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessor.java rename to suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetProcessor.java index 7ff9a190..e1ed0b90 100644 --- a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessor.java +++ b/suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetProcessor.java @@ -14,7 +14,7 @@ * limitations under the License. */ -package com.netflix.suro.queue; +package com.netflix.suro.input.thrift; import com.google.inject.Injector; import com.netflix.governator.configuration.PropertiesConfigurationProvider; @@ -23,7 +23,8 @@ import com.netflix.governator.guice.LifecycleInjector; import com.netflix.governator.lifecycle.LifecycleManager; import com.netflix.suro.connection.TestConnectionPool; -import com.netflix.suro.server.ServerConfig; +import com.netflix.suro.input.thrift.MessageSetProcessor; +import com.netflix.suro.input.thrift.ServerConfig; import com.netflix.suro.thrift.ResultCode; import com.netflix.suro.thrift.ServiceStatus; import com.netflix.suro.thrift.TMessageSet; diff --git a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetSerDe.java b/suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetSerDe.java similarity index 95% rename from suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetSerDe.java rename to suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetSerDe.java index cca1cdf2..a4fa62b8 100644 --- a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetSerDe.java +++ b/suro-server/src/test/java/com/netflix/suro/input/thrift/TestMessageSetSerDe.java @@ -14,9 +14,10 @@ * limitations under the License. */ -package com.netflix.suro.queue; +package com.netflix.suro.input.thrift; import com.netflix.suro.connection.TestConnectionPool; +import com.netflix.suro.input.thrift.MessageSetSerDe; import com.netflix.suro.message.Message; import com.netflix.suro.message.MessageSetReader; import com.netflix.suro.thrift.TMessageSet; diff --git a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessorManager.java b/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessorManager.java deleted file mode 100644 index eba10f59..00000000 --- a/suro-server/src/test/java/com/netflix/suro/queue/TestMessageSetProcessorManager.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Copyright 2013 Netflix, 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.netflix.suro.queue; - -import com.google.inject.Injector; -import com.netflix.governator.configuration.PropertiesConfigurationProvider; -import com.netflix.governator.guice.BootstrapBinder; -import com.netflix.governator.guice.BootstrapModule; -import com.netflix.governator.guice.LifecycleInjector; -import com.netflix.governator.lifecycle.LifecycleManager; -import com.netflix.suro.connection.TestConnectionPool; -import com.netflix.suro.thrift.ResultCode; -import org.junit.Test; - -import java.util.Properties; -import java.util.concurrent.TimeUnit; - -import static org.junit.Assert.assertEquals; - -public class TestMessageSetProcessorManager { - @Test - public void test() throws Exception { - final Properties props = new Properties(); - - Injector injector = LifecycleInjector.builder() - .withBootstrapModule(new BootstrapModule() { - @Override - public void configure(BootstrapBinder binder) { - binder.bindConfigurationProvider().toInstance(new PropertiesConfigurationProvider(props)); - } - }).build().createInjector(); - injector.getInstance(LifecycleManager.class).start(); - - MessageSetProcessor queue = injector.getInstance(MessageSetProcessor.class); - MessageSetProcessorManager manager = injector.getInstance(MessageSetProcessorManager.class); - manager.registerService(queue); - - assertEquals(queue.process(TestConnectionPool.createMessageSet(100)).getResultCode(), ResultCode.OK); - queue.poll(1, TimeUnit.SECONDS); - assertEquals(manager.getStatus(), MessageSetProcessorManager.OK); - - manager.stopTakingTraffic(); - assertEquals(queue.process(TestConnectionPool.createMessageSet(100)).getResultCode(), ResultCode.OTHER_ERROR); - - manager.startTakingTraffic(); - assertEquals(queue.process(TestConnectionPool.createMessageSet(100)).getResultCode(), ResultCode.OK); - } -} diff --git a/suro-server/src/test/java/com/netflix/suro/routing/TestMessageRouter.java b/suro-server/src/test/java/com/netflix/suro/routing/TestMessageRouter.java index 0aae61db..4ab626c8 100644 --- a/suro-server/src/test/java/com/netflix/suro/routing/TestMessageRouter.java +++ b/suro-server/src/test/java/com/netflix/suro/routing/TestMessageRouter.java @@ -32,9 +32,9 @@ import com.netflix.suro.message.MessageSetBuilder; import com.netflix.suro.message.SerDe; import com.netflix.suro.message.StringSerDe; -import com.netflix.suro.queue.MessageSetProcessor; +import com.netflix.suro.input.thrift.MessageSetProcessor; import com.netflix.suro.routing.RoutingMap.RoutingInfo; -import com.netflix.suro.server.ServerConfig; +import com.netflix.suro.input.thrift.ServerConfig; import com.netflix.suro.sink.Sink; import com.netflix.suro.sink.SinkManager; import org.junit.Assert; diff --git a/suro-server/src/test/java/com/netflix/suro/server/SuroServerExternalResource.java b/suro-server/src/test/java/com/netflix/suro/server/SuroServerExternalResource.java index 3a03fed0..f491fbb3 100644 --- a/suro-server/src/test/java/com/netflix/suro/server/SuroServerExternalResource.java +++ b/suro-server/src/test/java/com/netflix/suro/server/SuroServerExternalResource.java @@ -5,6 +5,7 @@ import com.netflix.suro.SuroPlugin; import com.netflix.suro.SuroServer; import com.netflix.suro.TestUtils; +import com.netflix.suro.input.DynamicPropertyInputConfigurator; import com.netflix.suro.routing.DynamicPropertyRoutingMapConfigurator; import com.netflix.suro.routing.TestMessageRouter; import com.netflix.suro.sink.DynamicPropertySinkConfigurator; @@ -29,10 +30,16 @@ public class SuroServerExternalResource extends ExternalResource { " }\n" + "}"; - private String mapDesc; + private String mapDesc="{}"; + private String inputConfig = "[\n" + + " {\n" + + " \"type\": \"thrift\"\n" + + " }\n" + + "]"; public SuroServerExternalResource() {} - public SuroServerExternalResource(String sinkDesc, String mapDesc) { + public SuroServerExternalResource(String inputConfig, String sinkDesc, String mapDesc) { + this.inputConfig = inputConfig; this.sinkDesc = sinkDesc; this.mapDesc = mapDesc; } @@ -46,6 +53,7 @@ protected void before() throws Exception { props.put("SuroServer.statusServerPort", Integer.toString(statusPort)); props.put("SuroServer.port", Integer.toString(serverPort)); props.put(DynamicPropertySinkConfigurator.SINK_PROPERTY, sinkDesc); + props.put(DynamicPropertyInputConfigurator.INPUT_CONFIG_PROPERTY, inputConfig); if (mapDesc != null) { props.put(DynamicPropertyRoutingMapConfigurator.ROUTING_MAP_PROPERTY, mapDesc); } diff --git a/suro-server/src/test/java/com/netflix/suro/server/TestStatusServer.java b/suro-server/src/test/java/com/netflix/suro/server/TestStatusServer.java index 69d32a05..c6fe309f 100644 --- a/suro-server/src/test/java/com/netflix/suro/server/TestStatusServer.java +++ b/suro-server/src/test/java/com/netflix/suro/server/TestStatusServer.java @@ -16,6 +16,7 @@ package com.netflix.suro.server; +import com.netflix.suro.input.InputManager; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpGet; import org.apache.http.impl.client.DefaultHttpClient; @@ -35,13 +36,13 @@ public class TestStatusServer { @Test public void connectionFailureShouldBeDetected() throws Exception { - suroServer.getInjector().getInstance(ThriftServer.class).shutdown(); + suroServer.getInjector().getInstance(InputManager.class).getInput("thrift").shutdown(); HttpResponse response = runQuery("surohealthcheck"); assertEquals(500, response.getStatusLine().getStatusCode()); - suroServer.getInjector().getInstance(ThriftServer.class).start(); + suroServer.getInjector().getInstance(InputManager.class).getInput("thrift").start(); } private HttpResponse runQuery(String path) throws IOException { diff --git a/suro-server/src/test/java/com/netflix/suro/sink/notice/TestNotice.java b/suro-server/src/test/java/com/netflix/suro/sink/notice/TestNotice.java index e956d283..6841a09b 100644 --- a/suro-server/src/test/java/com/netflix/suro/sink/notice/TestNotice.java +++ b/suro-server/src/test/java/com/netflix/suro/sink/notice/TestNotice.java @@ -16,62 +16,77 @@ package com.netflix.suro.sink.notice; +import com.amazonaws.auth.AWSCredentialsProvider; import com.amazonaws.services.sqs.AmazonSQSClient; -import com.amazonaws.services.sqs.model.GetQueueUrlRequest; -import com.amazonaws.services.sqs.model.GetQueueUrlResult; -import com.amazonaws.services.sqs.model.Message; -import com.amazonaws.services.sqs.model.ReceiveMessageRequest; -import com.amazonaws.services.sqs.model.ReceiveMessageResult; -import com.amazonaws.services.sqs.model.SendMessageRequest; -import com.amazonaws.services.sqs.model.SendMessageResult; +import com.amazonaws.services.sqs.model.*; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.BeanProperty; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Charsets; -import com.google.common.collect.Maps; import com.google.inject.AbstractModule; import com.google.inject.Guice; import com.google.inject.Injector; +import com.google.inject.Provider; import com.netflix.suro.SuroPlugin; +import com.netflix.suro.aws.PropertyAWSCredentialsProvider; import com.netflix.suro.jackson.DefaultObjectMapper; -import com.netflix.suro.sink.TestSinkManager.TestSink; +import com.netflix.suro.sink.TestSinkManager; import org.apache.commons.codec.binary.Base64; +import org.junit.Before; import org.junit.Test; import org.mockito.ArgumentCaptor; import java.io.IOException; import java.util.Arrays; -import java.util.Map; import static junit.framework.Assert.assertNull; import static junit.framework.TestCase.assertEquals; import static org.mockito.Matchers.any; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.*; public class TestNotice { - private static Injector injector = Guice.createInjector( - new SuroPlugin() { - @Override - protected void configure() { - this.addSinkType("TestSink", TestSink.class); - - this.addNoticeType(NoNotice.TYPE, NoNotice.class); - this.addNoticeType(QueueNotice.TYPE, QueueNotice.class); - this.addNoticeType(SQSNotice.TYPE, SQSNotice.class); + private Injector injector; + + @Before + public void setup() { + injector = Guice.createInjector( + new SuroPlugin() { + @Override + protected void configure() { + this.addSinkType("TestSink", TestSinkManager.TestSink.class); + + this.addNoticeType(NoNotice.TYPE, NoNotice.class); + this.addNoticeType(QueueNotice.TYPE, QueueNotice.class); + this.addNoticeType(SQSNotice.TYPE, SQSNotice.class); + } + }, + new AbstractModule() { + @Override + protected void configure() { + bind(ObjectMapper.class).to(DefaultObjectMapper.class); + bind(AWSCredentialsProvider.class).to(PropertyAWSCredentialsProvider.class); + + bind(AmazonSQSClient.class).toProvider(AmazonSQSClientProvider.class).asEagerSingleton(); + } } - }, - new AbstractModule() { - @Override - protected void configure() { - bind(ObjectMapper.class).to(DefaultObjectMapper.class); - } - } ); - + } + + public static class AmazonSQSClientProvider implements Provider { + + @Override + public AmazonSQSClient get() { + AmazonSQSClient client = mock(AmazonSQSClient.class); + doReturn(new SendMessageResult()).when(client).sendMessage(any(SendMessageRequest.class)); + + ReceiveMessageResult result = new ReceiveMessageResult(); + result.setMessages(Arrays.asList(new Message[]{new Message().withBody("receivedMessage")})); + doReturn(result).when(client).receiveMessage(any(ReceiveMessageRequest.class)); + doReturn(new GetQueueUrlResult().withQueueUrl("queueURL")).when(client).getQueueUrl(any(GetQueueUrlRequest.class)); + + return client; + } + } + @Test public void testQueue() throws IOException { String desc = "{\n" + @@ -155,28 +170,9 @@ public ArgumentCaptor getCaptor() { public SqsTest invoke() throws IOException { ObjectMapper mapper = injector.getInstance(DefaultObjectMapper.class); + AmazonSQSClient client = injector.getInstance(AmazonSQSClient.class); - AmazonSQSClient client = mock(AmazonSQSClient.class); - doReturn(new SendMessageResult()).when(client).sendMessage(any(SendMessageRequest.class)); - - ReceiveMessageResult result = new ReceiveMessageResult(); - result.setMessages(Arrays.asList(new Message[]{new Message().withBody("receivedMessage")})); - doReturn(result).when(client).receiveMessage(any(ReceiveMessageRequest.class)); - doReturn(new GetQueueUrlResult().withQueueUrl("queueURL")).when(client).getQueueUrl(any(GetQueueUrlRequest.class)); - - final Map injectables = Maps.newHashMap(); - injectables.put("sqsClient", client); - mapper.setInjectableValues(new InjectableValues() { - @Override - public Object findInjectableValue( - Object valueId, DeserializationContext ctxt, BeanProperty forProperty, Object beanInstance - ) { - return injectables.get(valueId); - } - }); - - queueNotice = mapper.readValue(desc, new TypeReference() { - }); + queueNotice = mapper.readValue(desc, new TypeReference() {}); queueNotice.init(); queueNotice.send("message"); captor = ArgumentCaptor.forClass(SendMessageRequest.class);