[
https://issues.apache.org/jira/browse/FLUME-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13236006#comment-13236006
]
Hari Shreedharan commented on FLUME-1036:
-----------------------------------------
Jarek - yes, this seems to be the same issue. For now just updating my
investigation here:
* I updated the PropertiesFileConfigurationProvider/DefaultLogicalNodeManager
to stop the old components. Some issues:
LifecycleSupervisor allows you to only switch to a desiredstate and wait for
MonitorRunnable to check the state within 3 seconds, which runs as a different
thread. So any stop/interrupt/join calls on any server-like process will block
only the MonitorRunnable thread, not the thread which changed the state. So it
is possible that the config provider thread will start the new components
before the old ones are dead - this is not the desired behavior. We absolutely
must wait for the old components to die before the new ones start up. So my
proposal:
*Add a method to LifecycleSupervisor - changeStateImmediately which will run
the monitorRunnable immediately and wait for it to die before returning. This
lets us wait till the state is changed and makes sure the component died.
> Reconfiguration of AVRO or NETCAT source causes port bind exception
> -------------------------------------------------------------------
>
> Key: FLUME-1036
> URL: https://issues.apache.org/jira/browse/FLUME-1036
> Project: Flume
> Issue Type: Bug
> Components: Sinks+Sources
> Affects Versions: v1.2.0
> Environment: [CentOS 6.2 64-bit]
> Reporter: Will McQueen
>
> This issue occurs with a source that has binds to a port, such as NETCAT or
> AVRO. In the steps below, I use AVRO as an example.
> Steps:
> 1. Run with a props file that includes an avro source, such as a props file
> with these contents:
> # a = agent
> # r = source
> # c = channel
> # k = sink
> a1.sources = r1
> a1.channels = c1
> a1.sinks = k1
> # ===SOURCES===
> a1.sources.r1.type = AVRO
> a1.sources.r1.bind = localhost
> a1.sources.r1.port = 1473
> a1.sources.r1.channels = c1
> # ===CHANNELS===
> a1.channels.c1.type = MEMORY
> # ===SINKS===
> a1.sinks.k1.type = LOGGER
> a1.sinks.k1.channel = c1
> 2. Run the flume node
> [will@localhost flume-1.2.0-incubating-SNAPSHOT]$ pwd
> /home/will/git/apache/flume/flume-1.2.0-incubating-SNAPSHOT
> [will@localhost flume-1.2.0-incubating-SNAPSHOT]$ bin/flume-ng node --conf
> conf --conf-file conf/a1.properties --name a1
> 3. Simulate a reconfig of a file by just touching the props file:
> [will@localhost flume-1.2.0-incubating-SNAPSHOT]$ touch conf/a1.properties
> At this point, an exception is thrown because an attempt is made to bind to
> the same port, even though the port number specified in config file didn't
> change (actually, nothing changed but the last modification time of the file).
> 2012-03-19 02:17:48,573 (lifecycleSupervisor-1-1) [ERROR -
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:210)]
> Unable to start EventDrivenSourceRunner: { source:AvroSource: {
> bindAddress:localhost port:1473 } } - Exception follows.
> org.jboss.netty.channel.ChannelException: Failed to bind to:
> localhost/127.0.0.1:1473
> at
> org.jboss.netty.bootstrap.ServerBootstrap.bind(ServerBootstrap.java:303)
> at org.apache.avro.ipc.NettyServer.<init>(NettyServer.java:85)
> at org.apache.avro.ipc.NettyServer.<init>(NettyServer.java:66)
> at org.apache.flume.source.AvroSource.start(AvroSource.java:124)
> at
> org.apache.flume.source.EventDrivenSourceRunner.start(EventDrivenSourceRunner.java:39)
> at
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:208)
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira