Skip to content

Commit

Permalink
FLUME-154: Scribe source.next() does not exit if it is closed
Browse files Browse the repository at this point in the history
  • Loading branch information
Jonathan Hsieh committed Aug 18, 2010
1 parent 4a3499e commit 6802fc8
Show file tree
Hide file tree
Showing 2 changed files with 84 additions and 38 deletions.
24 changes: 16 additions & 8 deletions src/java/com/cloudera/flume/handlers/scribe/ScribeEventSource.java
Expand Up @@ -51,17 +51,21 @@ public class ScribeEventSource extends ThriftServer implements EventSource,
static final Logger LOG = Logger.getLogger(ScribeEventSource.class);
final BlockingQueue<Event> pendingQueue = new LinkedBlockingQueue<Event>();

final static public String SCRIBE_CATEGORY = "scribe.category";
final static public String SCRIBE_CATEGORY = "scribe.category";
final AtomicBoolean running = new AtomicBoolean(false);
long startedTime = 0;
int port = 0;

final static Event DONE_EVENT = new EventImpl(new byte[0]);

/**
* Construct a scribe event source.
* @param port port the server will listen on
*
* @param port
* port the server will listen on
*/
public ScribeEventSource(int port) {
// turn off thrift strict read & write (respectively), otw legacy
// turn off thrift strict read & write (respectively), otw legacy
// thrift clients (ie scribe clients) won't be able to connect. This
// mimics what scribed does.
super(false, false);
Expand All @@ -80,6 +84,7 @@ public ScribeEventSource() {
public synchronized void close() throws IOException {
running.set(false);
this.stop();
pendingQueue.add(DONE_EVENT);
}

/**
Expand All @@ -88,7 +93,11 @@ public synchronized void close() throws IOException {
@Override
public Event next() throws IOException {
try {
return pendingQueue.take();
Event e = pendingQueue.take();
if (e == DONE_EVENT) {
return null;
}
return e;
} catch (InterruptedException e) {
LOG.error("ScribeEventSource was interrupted while waiting for an event",
e);
Expand All @@ -112,7 +121,7 @@ public synchronized void open() throws IOException {
protected TTransport acceptImpl() throws TTransportException {
return new TFramedTransport(super.acceptImpl());
}
});
});
running.set(true);
startedTime = Clock.unixTime();
} catch (TTransportException e) {
Expand Down Expand Up @@ -207,16 +216,15 @@ public void setOption(String key, String value) throws TException {
public void shutdown() throws TException {
throw new TException("shutdown not implemented!");
}

/**
* Builder takes one optional argument: the port to start on
*/
public static SourceBuilder builder() {
return new SourceBuilder() {
@Override
public EventSource build(String... argv) {
Preconditions.checkArgument(argv.length <= 1,
"usage: scribe[(port={"
Preconditions.checkArgument(argv.length <= 1, "usage: scribe[(port={"
+ FlumeConfiguration.DEFAULT_SCRIBE_SOURCE_PORT + "})]");
int port = FlumeConfiguration.get().getScribeSourcePort();
if (argv.length >= 1) {
Expand Down
Expand Up @@ -22,7 +22,10 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;

import org.apache.log4j.Logger;
import org.apache.thrift.TException;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TProtocol;
Expand All @@ -40,78 +43,113 @@
* Test the scribe event source which runs over thrift
*/
public class TestScribeSource {

final public static Logger LOG = Logger.getLogger(TestScribeSource.class);

/**
* Test that events can be sent and received, and that the correct metadata
* is extracted.
* Test that events can be sent and received, and that the correct metadata is
* extracted.
*/
@Test
public void testScribeEventSourceAPI() throws IOException, TException, InterruptedException {
ScribeEventSource src = new ScribeEventSource();
src.open();

public void testScribeEventSourceAPI() throws IOException, TException,
InterruptedException {
ScribeEventSource src = new ScribeEventSource();
src.open();

// Open the client connection
TTransport transport = new TSocket("localhost",
FlumeConfiguration.get().getScribeSourcePort());
TTransport transport = new TSocket("localhost", FlumeConfiguration.get()
.getScribeSourcePort());
// scribe clients used framed transports
transport = new TFramedTransport(transport);
// scribe clients do not use strict write
TProtocol protocol = new TBinaryProtocol(transport, false, false);
transport.open();
scribe.Client client = new scribe.Client(protocol);

// Note - there is a tiny possibility of a race here, which is why we retry
for (int i=0;i<3;++i) {
for (int i = 0; i < 3; ++i) {
if (client.getStatus() != fb_status.ALIVE) {
Thread.sleep(500);
} else {
break;
}
}
assertEquals("ScribeEventSource did not come up in time!",
fb_status.ALIVE, client.getStatus());
assertEquals("ScribeEventSource did not come up in time!", fb_status.ALIVE,
client.getStatus());

LogEntry l1 = new LogEntry("mycategory", "mymessage");
List<LogEntry> logs = new ArrayList<LogEntry>();
logs.add(l1);
client.Log(logs);

Event e = src.next();

src.close();

assertEquals("mymessage",
new String(e.getBody()), "mymessage");
assertEquals("mycategory",
new String(e.getAttrs().get(ScribeEventSource.SCRIBE_CATEGORY)));
}


assertEquals("mymessage", new String(e.getBody()), "mymessage");
assertEquals("mycategory", new String(e.getAttrs().get(
ScribeEventSource.SCRIBE_CATEGORY)));
}

@Test
public void testOpenClose() throws IOException, TException, InterruptedException {
public void testOpenClose() throws IOException, TException,
InterruptedException {
EventSource src = ScribeEventSource.builder().build("45872");
for (int i=0;i<10;++i) {
for (int i = 0; i < 10; ++i) {
src.open();
src.close();
}
src.open();

// Open the client connection
TTransport transport = new TSocket("localhost", 45872);
transport = new TFramedTransport(transport);
// scribe clients do not use strict write
TProtocol protocol = new TBinaryProtocol(transport, false, false);
transport.open();
scribe.Client client = new scribe.Client(protocol);

// Note - there is a tiny possibility of a race here, which is why we retry
for (int i=0;i<3;++i) {
for (int i = 0; i < 3; ++i) {
if (client.getStatus() != fb_status.ALIVE) {
Thread.sleep(500);
} else {
break;
}
}
assertEquals("ScribeEventSource did not come up in time!",
fb_status.ALIVE, client.getStatus());
assertEquals("ScribeEventSource did not come up in time!", fb_status.ALIVE,
client.getStatus());
src.close();
}

/**
* This test starts a scribe source and blocks on next in one thread and then
* attempt to close it from another. If the thread doesn't return the test
* will timeout and fail.
*/
@Test
public void testConcurrentClose() throws InterruptedException, IOException {
final CountDownLatch started = new CountDownLatch(1);
final CountDownLatch done = new CountDownLatch(1);
final ScribeEventSource src = new ScribeEventSource();

new Thread() {
@Override
public void run() {
try {
src.open();
started.countDown();
src.next();
done.countDown();
} catch (IOException e) {
LOG.error(e.getMessage(), e);
}
}
}.start();

assertTrue("Open timed out", started.await(5, TimeUnit.SECONDS));
src.close();
}
assertTrue("Next timed out", done.await(5, TimeUnit.SECONDS));

}
}

0 comments on commit 6802fc8

Please sign in to comment.