mirror of https://github.com/apache/druid.git
Merge pull request #1634 from metamx/IrcFirehoseFixes
Allow IrcFirehoseFactory to shutdown cleanly
This commit is contained in:
commit
6a0389297c
|
@ -39,11 +39,12 @@ import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.concurrent.LinkedBlockingQueue;
|
import java.util.concurrent.LinkedBlockingQueue;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p><b>Example code:</b></p>
|
* <p><b>Example code:</b></p>
|
||||||
* <pre>{@code
|
* <pre>{@code
|
||||||
*
|
* <p/>
|
||||||
* IrcFirehoseFactory factory = new IrcFirehoseFactory(
|
* IrcFirehoseFactory factory = new IrcFirehoseFactory(
|
||||||
* "wiki123",
|
* "wiki123",
|
||||||
* "irc.wikimedia.org",
|
* "irc.wikimedia.org",
|
||||||
|
@ -63,6 +64,7 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
private final String nick;
|
private final String nick;
|
||||||
private final String host;
|
private final String host;
|
||||||
private final List<String> channels;
|
private final List<String> channels;
|
||||||
|
private volatile boolean closed = false;
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public IrcFirehoseFactory(
|
public IrcFirehoseFactory(
|
||||||
|
@ -100,17 +102,21 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
final IRCApi irc = new IRCApiImpl(false);
|
final IRCApi irc = new IRCApiImpl(false);
|
||||||
final LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>> queue = new LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>>();
|
final LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>> queue = new LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>>();
|
||||||
|
|
||||||
irc.addListener(new VariousMessageListenerAdapter() {
|
irc.addListener(
|
||||||
@Override
|
new VariousMessageListenerAdapter()
|
||||||
public void onChannelMessage(ChannelPrivMsg aMsg)
|
{
|
||||||
{
|
@Override
|
||||||
try {
|
public void onChannelMessage(ChannelPrivMsg aMsg)
|
||||||
queue.put(Pair.of(DateTime.now(), aMsg));
|
{
|
||||||
} catch(InterruptedException e) {
|
try {
|
||||||
throw new RuntimeException("interrupted adding message to queue", e);
|
queue.put(Pair.of(DateTime.now(), aMsg));
|
||||||
|
}
|
||||||
|
catch (InterruptedException e) {
|
||||||
|
throw new RuntimeException("interrupted adding message to queue", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
);
|
||||||
});
|
|
||||||
|
|
||||||
log.info("connecting to irc server [%s]", host);
|
log.info("connecting to irc server [%s]", host);
|
||||||
irc.connect(
|
irc.connect(
|
||||||
|
@ -152,7 +158,7 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
public void onSuccess(IIRCState aObject)
|
public void onSuccess(IIRCState aObject)
|
||||||
{
|
{
|
||||||
log.info("irc connection to server [%s] established", host);
|
log.info("irc connection to server [%s] established", host);
|
||||||
for(String chan : channels) {
|
for (String chan : channels) {
|
||||||
log.info("Joining channel %s", chan);
|
log.info("Joining channel %s", chan);
|
||||||
irc.joinChannel(chan);
|
irc.joinChannel(chan);
|
||||||
}
|
}
|
||||||
|
@ -164,8 +170,10 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
log.error(e, "Unable to connect to irc server [%s]", host);
|
log.error(e, "Unable to connect to irc server [%s]", host);
|
||||||
throw new RuntimeException("Unable to connect to server", e);
|
throw new RuntimeException("Unable to connect to server", e);
|
||||||
}
|
}
|
||||||
});
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
closed = false;
|
||||||
|
|
||||||
return new Firehose()
|
return new Firehose()
|
||||||
{
|
{
|
||||||
|
@ -175,18 +183,26 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
public boolean hasMore()
|
public boolean hasMore()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
while(true) {
|
while (true) {
|
||||||
Pair<DateTime, ChannelPrivMsg> nextMsg = queue.take();
|
Pair<DateTime, ChannelPrivMsg> nextMsg = queue.poll(100, TimeUnit.MILLISECONDS);
|
||||||
|
if (closed) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (nextMsg == null) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
try {
|
try {
|
||||||
nextRow = firehoseParser.parse(nextMsg);
|
nextRow = firehoseParser.parse(nextMsg);
|
||||||
if(nextRow != null) return true;
|
if (nextRow != null) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
catch (IllegalArgumentException iae) {
|
catch (IllegalArgumentException iae) {
|
||||||
log.debug("ignoring invalid message in channel [%s]", nextMsg.rhs.getChannelName());
|
log.debug("ignoring invalid message in channel [%s]", nextMsg.rhs.getChannelName());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
catch(InterruptedException e) {
|
catch (InterruptedException e) {
|
||||||
Thread.interrupted();
|
Thread.interrupted();
|
||||||
throw new RuntimeException("interrupted retrieving elements from queue", e);
|
throw new RuntimeException("interrupted retrieving elements from queue", e);
|
||||||
}
|
}
|
||||||
|
@ -214,8 +230,13 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
||||||
@Override
|
@Override
|
||||||
public void close() throws IOException
|
public void close() throws IOException
|
||||||
{
|
{
|
||||||
log.info("disconnecting from irc server [%s]", host);
|
try {
|
||||||
irc.disconnect("");
|
log.info("disconnecting from irc server [%s]", host);
|
||||||
|
irc.disconnect("");
|
||||||
|
}
|
||||||
|
finally {
|
||||||
|
closed = true;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue