Thanks to visit codestin.com
Credit goes to github.com

Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -118,15 +118,17 @@ 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"
task (runServer, type: JavaExec, dependsOn: jar) {
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){
Expand Down
9 changes: 9 additions & 0 deletions suro-core/src/main/java/com/netflix/suro/SuroPlugin.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -33,6 +34,14 @@ public <T extends Sink> void addSinkType(String typeName, Class<T> sinkClass) {
bindings.addBinding().toInstance(new TypeHolder(typeName, sinkClass));
}

public <T extends SuroInput> void addInputType(String typeName, Class<T> inputClass) {
LOG.info("Adding inputType: " + typeName + " -> " + inputClass.getCanonicalName());

Multibinder<TypeHolder> bindings
= Multibinder.newSetBinder(binder(), TypeHolder.class);
bindings.addBinding().toInstance(new TypeHolder(typeName, inputClass));
}

public <T extends RemotePrefixFormatter> void addRemotePrefixFormatterType(String typeName, Class<T> remotePrefixFormatterClass) {
LOG.info("Adding remotePrefixFormatterType: " + typeName + " -> " + remotePrefixFormatterClass.getCanonicalName());

Expand Down
13 changes: 13 additions & 0 deletions suro-core/src/main/java/com/netflix/suro/input/SuroInput.java
Original file line number Diff line number Diff line change
@@ -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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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<SuroInput> inputList = new ArrayList<SuroInput>();

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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -32,7 +32,7 @@
*
* @author jbae
*/
@LazySingleton
@Singleton
public class MessageRouter {
private static final Logger log = LoggerFactory.getLogger(MessageRouter.class);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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<String, Sink> sinkMap = Maps.newConcurrentMap();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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" +
Expand Down Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
@@ -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<byte[], byte[]> 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<String, List<KafkaStream<byte[], byte[]>>> streams = connector.createMessageStreams(ImmutableMap.of(topic, 1));

final List<KafkaStream<byte[], byte[]>> 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();
}
}
Loading