mirror of https://github.com/apache/nifi.git
NIFI-1436 This closes #189. Combining stop() and close() into a single method to simplify, and adding checks on stopped flag in the run method of SocketChannelDispatcher and DatagramChannelDispatcher to ensure the run() method exits as soon as possible upon close() being called
NIFI-1436 Adding synchronization on keys set in close() method based on Selector JavaDoc Signed-off-by: joewitt <joewitt@apache.org>
This commit is contained in:
parent
925138b6c4
commit
b7f7e6ed80
|
@ -212,7 +212,6 @@ public abstract class AbstractListenEventProcessor<E extends Event> extends Abst
|
|||
@OnUnscheduled
|
||||
public void onUnscheduled() {
|
||||
if (dispatcher != null) {
|
||||
dispatcher.stop();
|
||||
dispatcher.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,11 +38,6 @@ public interface ChannelDispatcher extends Runnable {
|
|||
*/
|
||||
int getPort();
|
||||
|
||||
/**
|
||||
* Stops the main dispatcher thread.
|
||||
*/
|
||||
void stop();
|
||||
|
||||
/**
|
||||
* Closes all listeners and stops all handler threads.
|
||||
*/
|
||||
|
|
|
@ -65,8 +65,10 @@ public class DatagramChannelDispatcher<E extends Event<DatagramChannel>> impleme
|
|||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
stopped = false;
|
||||
datagramChannel = DatagramChannel.open();
|
||||
datagramChannel.configureBlocking(false);
|
||||
|
||||
if (maxBufferSize > 0) {
|
||||
datagramChannel.setOption(StandardSocketOptions.SO_RCVBUF, maxBufferSize);
|
||||
final int actualReceiveBufSize = datagramChannel.getOption(StandardSocketOptions.SO_RCVBUF);
|
||||
|
@ -87,9 +89,11 @@ public class DatagramChannelDispatcher<E extends Event<DatagramChannel>> impleme
|
|||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
// if stopped the selector could already be closed which would result in a ClosedSelectorException
|
||||
if (selected > 0 && !stopped) {
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()) {
|
||||
// if stopped we don't want to modify the keys because close() may still be in progress
|
||||
while (selectorKeys.hasNext() && !stopped) {
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()) {
|
||||
|
@ -140,14 +144,12 @@ public class DatagramChannelDispatcher<E extends Event<DatagramChannel>> impleme
|
|||
return datagramChannel == null ? 0 : datagramChannel.socket().getLocalPort();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
selector.wakeup();
|
||||
stopped = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
stopped = true;
|
||||
if (selector != null) {
|
||||
selector.wakeup();
|
||||
}
|
||||
IOUtils.closeQuietly(selector);
|
||||
IOUtils.closeQuietly(datagramChannel);
|
||||
}
|
||||
|
|
|
@ -91,7 +91,8 @@ public class SocketChannelDispatcher<E extends Event<SocketChannel>> implements
|
|||
|
||||
@Override
|
||||
public void open(final int port, int maxBufferSize) throws IOException {
|
||||
this.executor = Executors.newFixedThreadPool(maxConnections);
|
||||
stopped = false;
|
||||
executor = Executors.newFixedThreadPool(maxConnections);
|
||||
|
||||
final ServerSocketChannel serverSocketChannel = ServerSocketChannel.open();
|
||||
serverSocketChannel.configureBlocking(false);
|
||||
|
@ -114,9 +115,11 @@ public class SocketChannelDispatcher<E extends Event<SocketChannel>> implements
|
|||
while (!stopped) {
|
||||
try {
|
||||
int selected = selector.select();
|
||||
if (selected > 0){
|
||||
// if stopped the selector could already be closed which would result in a ClosedSelectorException
|
||||
if (selected > 0 && !stopped){
|
||||
Iterator<SelectionKey> selectorKeys = selector.selectedKeys().iterator();
|
||||
while (selectorKeys.hasNext()){
|
||||
// if stopped we don't want to modify the keys because close() may still be in progress
|
||||
while (selectorKeys.hasNext() && !stopped) {
|
||||
SelectionKey key = selectorKeys.next();
|
||||
selectorKeys.remove();
|
||||
if (!key.isValid()){
|
||||
|
@ -196,28 +199,34 @@ public class SocketChannelDispatcher<E extends Event<SocketChannel>> implements
|
|||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
stopped = true;
|
||||
selector.wakeup();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait a while for existing tasks to terminate
|
||||
if (!executor.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
// (Re-)Cancel if current thread also interrupted
|
||||
executor.shutdownNow();
|
||||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
stopped = true;
|
||||
if (selector != null) {
|
||||
selector.wakeup();
|
||||
}
|
||||
for(SelectionKey key : selector.keys()){
|
||||
IOUtils.closeQuietly(key.channel());
|
||||
|
||||
if (executor != null) {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait a while for existing tasks to terminate
|
||||
if (!executor.awaitTermination(1000L, TimeUnit.MILLISECONDS)) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
} catch (InterruptedException ie) {
|
||||
// (Re-)Cancel if current thread also interrupted
|
||||
executor.shutdownNow();
|
||||
// Preserve interrupt status
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
if (selector != null) {
|
||||
synchronized (selector.keys()) {
|
||||
for (SelectionKey key : selector.keys()) {
|
||||
IOUtils.closeQuietly(key.channel());
|
||||
}
|
||||
}
|
||||
}
|
||||
IOUtils.closeQuietly(selector);
|
||||
}
|
||||
|
|
|
@ -309,7 +309,6 @@ public class ListenSyslog extends AbstractSyslogProcessor {
|
|||
@OnUnscheduled
|
||||
public void onUnscheduled() {
|
||||
if (channelDispatcher != null) {
|
||||
channelDispatcher.stop();
|
||||
channelDispatcher.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -165,7 +165,6 @@ public class TestRELPSocketChannelHandler {
|
|||
|
||||
} finally {
|
||||
// stop the dispatcher thread and ensure we shut down handler threads
|
||||
dispatcher.stop();
|
||||
dispatcher.close();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue