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.UUID;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* <p><b>Example code:</b></p>
|
||||
* <pre>{@code
|
||||
*
|
||||
* <p/>
|
||||
* IrcFirehoseFactory factory = new IrcFirehoseFactory(
|
||||
* "wiki123",
|
||||
* "irc.wikimedia.org",
|
||||
|
@ -63,6 +64,7 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
|||
private final String nick;
|
||||
private final String host;
|
||||
private final List<String> channels;
|
||||
private volatile boolean closed = false;
|
||||
|
||||
@JsonCreator
|
||||
public IrcFirehoseFactory(
|
||||
|
@ -100,17 +102,21 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
|||
final IRCApi irc = new IRCApiImpl(false);
|
||||
final LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>> queue = new LinkedBlockingQueue<Pair<DateTime, ChannelPrivMsg>>();
|
||||
|
||||
irc.addListener(new VariousMessageListenerAdapter() {
|
||||
irc.addListener(
|
||||
new VariousMessageListenerAdapter()
|
||||
{
|
||||
@Override
|
||||
public void onChannelMessage(ChannelPrivMsg aMsg)
|
||||
{
|
||||
try {
|
||||
queue.put(Pair.of(DateTime.now(), aMsg));
|
||||
} catch(InterruptedException e) {
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
throw new RuntimeException("interrupted adding message to queue", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
);
|
||||
|
||||
log.info("connecting to irc server [%s]", host);
|
||||
irc.connect(
|
||||
|
@ -164,8 +170,10 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
|||
log.error(e, "Unable to connect to irc server [%s]", host);
|
||||
throw new RuntimeException("Unable to connect to server", e);
|
||||
}
|
||||
});
|
||||
}
|
||||
);
|
||||
|
||||
closed = false;
|
||||
|
||||
return new Firehose()
|
||||
{
|
||||
|
@ -176,10 +184,18 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
|||
{
|
||||
try {
|
||||
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 {
|
||||
nextRow = firehoseParser.parse(nextMsg);
|
||||
if(nextRow != null) return true;
|
||||
if (nextRow != null) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
catch (IllegalArgumentException iae) {
|
||||
log.debug("ignoring invalid message in channel [%s]", nextMsg.rhs.getChannelName());
|
||||
|
@ -214,9 +230,14 @@ public class IrcFirehoseFactory implements FirehoseFactory<IrcInputRowParser>
|
|||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
try {
|
||||
log.info("disconnecting from irc server [%s]", host);
|
||||
irc.disconnect("");
|
||||
}
|
||||
finally {
|
||||
closed = true;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue