flume-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Juhani Connolly <juhani_conno...@cyberagent.co.jp>
Subject Re: Problems with failover configuration
Date Mon, 25 Jun 2012 01:42:15 GMT
As Hari said, the priorities are backwards(this is an implementation 
detail... When nothing is assigned, the defaults are assigned to 
negative values to keep their priorities lower).

Another thing to note is that the failover sink processor will retry 
failed sinks regularly, gradually taking longer breaks between attempts. 
Looking at your logs it seemed like the behaviour was correct(though 
backwards because of the priorities set to the two sinks), with error at 
56:59, 57:02, 57:09... I'm going to presume that the gap to the next set 
of errors was even further along(at least 8 seconds later)?

On 06/24/2012 09:24 PM, Stern, Mark wrote:
> I am trying to set up flume-ng with a failover configuration. However, I am getting
> a continual stream of error messages because the machine on the backup route
> is not available. I do not understand why this is such a problem - the main route
> is available and working. Please help. I am using Flume version 1.1.0 (from latest
> Cloudera RPM) I do not want to have to upgrade (because that means building from
> source) unless you are sure that will fix the problem.
>
> Thanks,
>
> Mark Stern
>
> Here is my configuration:
>
> # Define jdbc channels on all the agents
> agent_log.channels.ch_log.type = jdbc
>
> # Define an Exec source called exec-source on agent_log
> # Connect it to channel ch_log.
> agent_log.sources.exec-source.channels = ch_log
> agent_log.sources.exec-source.type = exec
> agent_log.sources.exec-source.command = /var/run/flume-ng/bin/mygrep
>
> # Define an Avro sink on agent_log for forwarding to agent_router (main)
> agent_log.sinks.avro-sink_log_main.channel = ch_log
> agent_log.sinks.avro-sink_log_main.type = avro
> agent_log.sinks.avro-sink_log_main.hostname = isk-vsrv701
> agent_log.sinks.avro-sink_log_main.port = 41414
> agent_log.sinks.avro-sink_log_main.batch-size = 20
>
> # Define an Avro sink on agent_log for forwarding to agent_router (backup)
> agent_log.sinks.avro-sink_log_backup.channel = ch_log
> agent_log.sinks.avro-sink_log_backup.type = avro
> agent_log.sinks.avro-sink_log_backup.hostname = isk-vsrv710
> agent_log.sinks.avro-sink_log_backup.port = 41414
> agent_log.sinks.avro-sink_log_backup.batch-size = 20
>
> # Specify priorities for the sinks on agent_log
> agent_log.sinkgroups.group_log.sinks = avro-sink_log_main avro-sink_log_backup
> agent_log.sinkgroups.group_log.processor.type = failover
> agent_log.sinkgroups.group_log.processor.priority.avro-sink_log_main = 1
> agent_log.sinkgroups.group_log.processor.priority.avro-sink_log_backup = 2
>
> # Finally, now that we've defined all of our components, tell
> # the agents which ones to use.
> agent_log.channels = ch_log
> agent_log.sources = exec-source
> agent_log.sinks = avro-sink_log_main avro-sink_log_backup
> agent_log.sinkgroups = group_log
>
> Here is the start of the output:
>
> Warning: No configuration directory set! Use --conf<dir>  to override.
> Info: Including Hadoop libraries found via (/usr/bin/hadoop) for HDFS access
> Info: Excluding /usr/lib/hadoop/lib/slf4j-api-1.6.1.jar from classpath
> Info: Excluding /usr/lib/hadoop/lib/slf4j-log4j12-1.6.1.jar from classpath
> Info: Excluding /usr/lib/hadoop-hdfs/lib/slf4j-api-1.6.1.jar from classpath
> + exec /usr/lib/jvm/jre-openjdk/bin/java -Xmx20m -cp '/usr/lib/flume-ng/lib/*:/etc/hadoop/conf:/usr/lib/hadoop/lib/activation-1.1.jar:/usr/lib/hadoop/lib/asm-3.2.jar:/usr/lib/hadoop/lib/aspectjrt-1.6.5.jar:/usr/lib/hadoop/lib/avro-1.5.4.jar:/usr/lib/hadoop/lib/commons-beanutils-1.7.0.jar:/usr/lib/hadoop/lib/commons-beanutils-core-1.8.0.jar:/usr/lib/hadoop/lib/commons-cli-1.2.jar:/usr/lib/hadoop/lib/commons-codec-1.4.jar:/usr/lib/hadoop/lib/commons-collections-3.2.1.jar:/usr/lib/hadoop/lib/commons-configuration-1.6.jar:/usr/lib/hadoop/lib/commons-digester-1.8.jar:/usr/lib/hadoop/lib/commons-el-1.0.jar:/usr/lib/hadoop/lib/commons-httpclient-3.1.jar:/usr/lib/hadoop/lib/commons-io-2.1.jar:/usr/lib/hadoop/lib/commons-lang-2.5.jar:/usr/lib/hadoop/lib/commons-logging-1.1.1.jar:/usr/lib/hadoop/lib/commons-logging-api-1.1.jar:/usr/lib/hadoop/lib/commons-math-2.1.jar:/usr/lib/hadoop/lib/commons-net-3.1.jar:/usr/lib/hadoop/lib/core-3.1.1.jar:/usr/lib/hadoop/lib/guava-11.0.2.jar:/usr/lib/hadoop/lib/jackson-core-asl-1.8.8.jar:/usr/lib/hadoop/lib/jackson-jaxrs-1.8.8.jar:/usr/lib/hadoop/lib/jackson-mapper-asl-1.8.8.jar:/usr/lib/hadoop/lib/jackson-xc-1.8.8.jar:/usr/lib/hadoop/lib/jasper-compiler-5.5.23.jar:/usr/lib/hadoop/lib/jasper-runtime-5.5.23.jar:/usr/lib/hadoop/lib/jaxb-api-2.2.2.jar:/usr/lib/hadoop/lib/jaxb-impl-2.2.3-1.jar:/usr/lib/hadoop/lib/jersey-core-1.8.jar:/usr/lib/hadoop/lib/jersey-json-1.8.jar:/usr/lib/hadoop/lib/jersey-server-1.8.jar:/usr/lib/hadoop/lib/jets3t-0.6.1.jar:/usr/lib/hadoop/lib/jettison-1.1.jar:/usr/lib/hadoop/lib/jetty-6.1.26.cloudera.1.jar:/usr/lib/hadoop/lib/jetty-util-6.1.26.cloudera.1.jar:/usr/lib/hadoop/lib/jline-0.9.94.jar:/usr/lib/hadoop/lib/jsch-0.1.42.jar:/usr/lib/hadoop/lib/json-simple-1.1.jar:/usr/lib/hadoop/lib/jsp-api-2.1.jar:/usr/lib/hadoop/lib/jsr305-1.3.9.jar:/usr/lib/hadoop/lib/kfs-0.3.jar:/usr/lib/hadoop/lib/log4j-1.2.15.jar:/usr/lib/hadoop/lib/native:/usr/lib/hadoop/lib/oro-2.0.8.jar:/usr/lib/hadoop/lib/paranamer-2.3.jar:/usr/lib/hadoop/lib/protobuf-java-2.4.0a.jar:/usr/lib/hadoop/lib/servlet-api-2.5.jar:/usr/lib/hadoop/lib/snappy-java-1.0.3.2.jar:/usr/lib/hadoop/lib/stax-api-1.0.1.jar:/usr/lib/hadoop/lib/xmlenc-0.52.jar:/usr/lib/hadoop/lib/zookeeper-3.4.3-cdh4.0.0.jar:/usr/lib/hadoop/.//bin:/usr/lib/hadoop/.//cloudera:/usr/lib/hadoop/.//etc:/usr/lib/hadoop/.//hadoop-annotations-2.0.0-cdh4.0.0.jar:/usr/lib/hadoop/.//hadoop-annotations.jar:/usr/lib/hadoop/.//hadoop-auth-2.0.0-cdh4.0.0.jar:/usr/lib/hadoop/.//hadoop-auth.jar:/usr/lib/hadoop/.//hadoop-common-2.0.0-cdh4.0.0.jar:/usr/lib/hadoop/.//hadoop-common-2.0.0-cdh4.0.0-tests.jar:/usr/lib/hadoop/.//hadoop-common.jar:/usr/lib/hadoop/.//lib:/usr/lib/hadoop/.//libexec:/usr/lib/hadoop/.//sbin:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/avro-1.5.4.jar:/usr/lib/hadoop-hdfs/lib/commons-daemon-1.0.3.jar:/usr/lib/hadoop-hdfs/lib/commons-logging-1.1.1.jar:/usr/lib/hadoop-hdfs/lib/jackson-core-asl-1.8.8.jar:/usr/lib/hadoop-hdfs/lib/jackson-mapper-asl-1.8.8.jar:/usr/lib/hadoop-hdfs/lib/jline-0.9.94.jar:/usr/lib/hadoop-hdfs/lib/log4j-1.2.15.jar:/usr/lib/hadoop-hdfs/lib/paranamer-2.3.jar:/usr/lib/hadoop-hdfs/lib/protobuf-java-2.4.0a.jar:/usr/lib/hadoop-hdfs/lib/snappy-java-1.0.3.2.jar:/usr/lib/hadoop-hdfs/lib/zookeeper-3.4.3-cdh4.0.0.jar:/usr/lib/hadoop-hdfs/.//bin:/usr/lib/hadoop-hdfs/.//cloudera:/usr/lib/hadoop-hdfs/.//hadoop-hdfs-2.0.0-cdh4.0.0.jar:/usr/lib/hadoop-hdfs/.//hadoop-hdfs-2.0.0-cdh4.0.0-tests.jar:/usr/lib/hadoop-hdfs/.//hadoop-hdfs.jar:/usr/lib/hadoop-hdfs/.//lib:/usr/lib/hadoop-hdfs/.//sbin:/usr/lib/hadoop-hdfs/.//webapps:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-0.20-mapreduce/.//*'
-Djava.library.path=://usr/lib/hadoop/lib/native org.apache.flume.node.Application -f /etc/flume-ng/conf/flume.conf
-n agent_log
> 12/06/24 11:56:58 INFO lifecycle.LifecycleSupervisor: Starting lifecycle supervisor 1
> 12/06/24 11:56:58 INFO node.FlumeNode: Flume node starting - agent_log
> 12/06/24 11:56:58 INFO nodemanager.DefaultLogicalNodeManager: Node manager starting
> 12/06/24 11:56:58 INFO lifecycle.LifecycleSupervisor: Starting lifecycle supervisor 9
> 12/06/24 11:56:58 INFO properties.PropertiesFileConfigurationProvider: Configuration
provider starting
> 12/06/24 11:56:58 INFO properties.PropertiesFileConfigurationProvider: Reloading configuration
file:/etc/flume-ng/conf/flume.conf
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_main
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_backup
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_main
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_backup
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:roll-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Added sinks: roll-sink Agent: agent_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_backup
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_main
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Added sinks: avro-sink_log_main avro-sink_log_backup
Agent: agent_log
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_backup
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Added sinks: avro-sink_fc avro-sink_ag
Agent: agent_router
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:roll-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:roll-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Added sinks: hdfs-sink Agent: agent_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_ag
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_main
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:roll-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_main
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_log_backup
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:hdfs-sink
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Processing:avro-sink_fc
> 12/06/24 11:56:58 INFO conf.FlumeConfiguration: Post-validation flume configuration contains
configuration  for agents: [agent_router, agent_fc, agent_ag, agent_log]
> 12/06/24 11:56:58 INFO properties.PropertiesFileConfigurationProvider: Creating channels
> 12/06/24 11:56:58 WARN impl.JdbcChannelProviderImpl: No connection URL specified. Using
embedded derby database instance.
> 12/06/24 11:56:58 WARN impl.JdbcChannelProviderImpl: Overriding values for - driver:
org.apache.derby.jdbc.EmbeddedDriver, user: saconnectUrl: jdbc:derby:/var/run/flume-ng/.flume/jdbc-channel/db;create=true,
jdbc properties file: null, dbtype: DERBY
> Sun Jun 24 11:56:58 UTC 2012 Thread[conf-file-poller-0,5,main] java.io.FileNotFoundException:
derby.log (Permission denied)
> ----------------------------------------------------------------
> Sun Jun 24 11:56:59 UTC 2012:
> Booting Derby version The Apache Software Foundation - Apache Derby - 10.8.2.2 - (1181258):
instance a816c00e-0138-1e5a-1ee3-000000e8b4d0
> on database directory /var/run/flume-ng/.flume/jdbc-channel/db  with class loader sun.misc.Launcher$AppClassLoader@553f5d07
> Loaded from file:/usr/lib/flume-ng/lib/derby-10.8.2.2.jar
> java.vendor=Sun Microsystems Inc.
> java.runtime.version=1.6.0_22-b22
> user.dir=/usr2/mstern
> derby.system.home=null
> Database Class Loader started - derby.database.classpath=''
> 12/06/24 11:56:59 WARN impl.JdbcChannelProviderImpl: JDBC channel will operate without
a capacity limit.
> 12/06/24 11:56:59 INFO jdbc.JdbcChannel: JDBC Channel initialized: ch_log
> 12/06/24 11:56:59 INFO properties.PropertiesFileConfigurationProvider: created channel
ch_log
> 12/06/24 11:56:59 INFO sink.DefaultSinkFactory: Creating instance of sink avro-sink_log_main
typeavro
> 12/06/24 11:56:59 INFO sink.DefaultSinkFactory: Creating instance of sink avro-sink_log_backup
typeavro
> 12/06/24 11:56:59 INFO nodemanager.DefaultLogicalNodeManager: Starting new configuration:{
sourceRunners:{exec-source=EventDrivenSourceRunner: { source:org.apache.flume.source.ExecSource@4c6504bc
}} sinkRunners:{group_log=SinkRunner: { policy:org.apache.flume.sink.FailoverSinkProcessor@7c19f9d2
counterGroup:{ name:null counters:{} } }} channels:{ch_log=org.apache.flume.channel.jdbc.JdbcChannel@d6089a5}
}
> 12/06/24 11:56:59 INFO nodemanager.DefaultLogicalNodeManager: Starting Channel ch_log
> 12/06/24 11:56:59 INFO nodemanager.DefaultLogicalNodeManager: Starting Sink group_log
> 12/06/24 11:56:59 INFO sink.AvroSink: Avro sink starting
> 12/06/24 11:56:59 INFO nodemanager.DefaultLogicalNodeManager: Starting Source exec-source
> 12/06/24 11:56:59 INFO source.ExecSource: Exec source starting with command:/var/run/flume-ng/bin/mygrep
> 12/06/24 11:56:59 INFO sink.AvroSink: Avro sink starting
> 12/06/24 11:56:59 ERROR api.NettyAvroRpcClient: RPC connection error :
> java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.start(AvroSink.java:188)
>          at org.apache.flume.sink.AbstractSinkProcessor.start(AbstractSinkProcessor.java:41)
>          at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>          at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:228)
>          at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>          at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
>          at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          ... 3 more
> 12/06/24 11:56:59 ERROR sink.AvroSink: Unable to create avro client using hostname:isk-vsrv710,
port:41414, batchSize: 20. Exception follows.
> org.apache.flume.FlumeException: RPC connection error. Exception follows.
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:126)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.start(AvroSink.java:188)
>          at org.apache.flume.sink.AbstractSinkProcessor.start(AbstractSinkProcessor.java:41)
>          at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>          at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:228)
>          at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
>          at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351)
>          at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:165)
>          at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:267)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          ... 16 more
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          ... 3 more
> 12/06/24 11:56:59 ERROR api.NettyAvroRpcClient: RPC connection error :
> java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.verifyConnection(AvroSink.java:176)
>          at org.apache.flume.sink.AvroSink.process(AvroSink.java:226)
>          at org.apache.flume.sink.FailoverSinkProcessor.process(FailoverSinkProcessor.java:182)
>          at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          ... 1 more
> 12/06/24 11:56:59 WARN impl.JdbcTransactionImpl: Marking transaction for rollback
> 12/06/24 11:56:59 INFO impl.JdbcTransactionImpl: Attempting transaction roll-back
> 12/06/24 11:56:59 WARN sink.FailoverSinkProcessor: Sink avro-sink_log_backup failed and
has been sent to failover list
> org.apache.flume.EventDeliveryException: RPC connection error. Exception follows.
>          at org.apache.flume.sink.AvroSink.process(AvroSink.java:264)
>          at org.apache.flume.sink.FailoverSinkProcessor.process(FailoverSinkProcessor.java:182)
>          at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: org.apache.flume.FlumeException: RPC connection error. Exception follows.
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:126)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.verifyConnection(AvroSink.java:176)
>          at org.apache.flume.sink.AvroSink.process(AvroSink.java:226)
>          ... 3 more
> Caused by: java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          ... 9 more
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          ... 1 more
> 12/06/24 11:57:02 ERROR api.NettyAvroRpcClient: RPC connection error :
> java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.verifyConnection(AvroSink.java:176)
>          at org.apache.flume.sink.AvroSink.process(AvroSink.java:226)
>          at org.apache.flume.sink.FailoverSinkProcessor.process(FailoverSinkProcessor.java:162)
>          at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
>          at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          ... 1 more
> 12/06/24 11:57:02 WARN impl.JdbcTransactionImpl: Marking transaction for rollback
> 12/06/24 11:57:02 INFO impl.JdbcTransactionImpl: Attempting transaction roll-back
> 12/06/24 11:57:09 ERROR api.NettyAvroRpcClient: RPC connection error :
> java.io.IOException: Error connecting to isk-vsrv710/10.231.148.171:41414
>          at org.apache.avro.ipc.NettyTransceiver.getChannel(NettyTransceiver.java:250)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:199)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:148)
>          at org.apache.avro.ipc.NettyTransceiver.<init>(NettyTransceiver.java:116)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:120)
>          at org.apache.flume.api.NettyAvroRpcClient.connect(NettyAvroRpcClient.java:109)
>          at org.apache.flume.api.NettyAvroRpcClient.<init>(NettyAvroRpcClient.java:94)
>          at org.apache.flume.api.RpcClientFactory.getDefaultInstance(RpcClientFactory.java:131)
>          at org.apache.flume.sink.AvroSink.createConnection(AvroSink.java:148)
>          at org.apache.flume.sink.AvroSink.verifyConnection(AvroSink.java:176)
>          at org.apache.flume.sink.AvroSink.process(AvroSink.java:226)
>          at org.apache.flume.sink.FailoverSinkProcessor.process(FailoverSinkProcessor.java:162)
>          at org.apache.flume.SinkRunner$PollingRunner.run(SinkRunner.java:147)
>          at java.lang.Thread.run(Thread.java:679)
> Caused by: java.net.ConnectException: Connection refused
>          at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>          at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:592)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.connect(NioClientSocketPipelineSink.java:384)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.processSelectedKeys(NioClientSocketPipelineSink.java:354)
>          at org.jboss.netty.channel.socket.nio.NioClientSocketPipelineSink$Boss.run(NioClientSocketPipelineSink.java:276)
>          at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
>          at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:44)
>          at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
>          at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
>          ... 1 more
>
> 	
> 	
>



Mime
View raw message