You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flume.apache.org by iain wright <ia...@gmail.com> on 2012/10/08 21:29:38 UTC

Flume HbaseSink ZK woes

We're having some trouble with the Flume & the HbaseSink. Seems we cannot
hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo both
look fine, Don't think we are hitting our
hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions
and I believe it defaults to 2k. I can establish new sessions using the CLI
from the same server and idle there for 10 minutes without getting dropped.

Flume & zookeeper logs below, using the same hadoop & hbase directories
from our regionserver's.

*Searched the list, this user appeared to have the same problem, not sure
how he fixed it though:*
http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E

*startup cmd
*/app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c ./conf -f
conf/brian.properties  -Dflume.root.logger=INFO,console*
*
*flume-env.sh*
$ cat conf/flume-env.sh

# Licensed to the Apache Software Foundation (ASF) under one
# or more contributor license agreements.  See the NOTICE file
# distributed with this work for additional information
# regarding copyright ownership.  The ASF licenses this file
# to you under the Apache License, Version 2.0 (the
# "License"); you may not use this file except in compliance
# with the License.  You may obtain a copy of the License at
#
#     http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.

# If this file is placed at FLUME_CONF_DIR/flume-env.sh, it will be sourced
# during Flume startup.

# Enviroment variables can be set here.

#JAVA_HOME=/usr/lib/jvm/java-6-sun

# Give Flume more memory and pre-allocate, enable remote monitoring via JMX
#JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"

# Note that the Flume conf directory is always included in the classpath.
#FLUME_CLASSPATH="/app/flume/lib"

FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"

HBASE_HOME="/app/hbase-0.94.0"

HADOOP_HOME="/app/hadoop-1.0.1"


*config*
$ cat conf/brian.properties
#example.conf: A single-node Flume configuration

# Name the components on this agent
agent1.sources = source1
agent1.sinks = sink1
agent1.channels = channel1

# Describe/configure source1
agent1.sources.source1.type = exec
agent1.sources.source1.command = tail -F /tank/log_dev.log
agent1.sources.source1.batchSize = 1
# Describe sink1
#agent1.sinks.sink1.type = logger
agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
agent1.sinks.sink1.table = brian_test
agent1.sinks.sink1.columnFamily = f1
#agent1.sinks.sink1.serializer =
org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
#agent1.sinks.sink1.serializer =
org.apache.flume.sink.hbase.SimpleHbaseEventSerializer


# Use a channel which buffers events in memory
agent1.channels.channel1.type = memory
agent1.channels.channel1.capacity = 1000
agent1.channels.channel1.transactionCapactiy = 100

# Bind the source and sink to the channel
agent1.sources.source1.channels = channel1
agent1.sinks.sink1.channel = channel1


*flume console log*
2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:java.io.tmpdir=/var/tmp/
2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:java.compiler=<NA>
2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:os.name=FreeBSD
2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:os.arch=amd64
2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:os.version=9.0-RELEASE
2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:user.name=czhang
2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:user.home=/users/czhang
2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO -
org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating
client connection,
connectString=hbasemaster0.hadoop.domain.com:2181sessionTimeout=180000
watcher=hconnection
2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)]
The identifier of this process is 12494@app14.app.domain.com
2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
Opening socket connection to server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
authenticate using SASL (unknown error)
2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
Socket connection established to
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
Session establishment complete on server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
0x138a0620045bbce, negotiated timeout = 180000
2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
Client session timed out, have not heard from server in 120001ms for
sessionid 0x138a0620045bbce, closing socket connection and attempting
reconnect
2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
Possibly transient ZooKeeper exception:
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO -
org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
Sleeping 2000ms before retry #1...
2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
Opening socket connection to server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
authenticate using SASL (unknown error)
2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
Socket connection established to
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
Session establishment complete on server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
0x138a0620045bbce, negotiated timeout = 180000
2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
Client session timed out, have not heard from server in 120000ms for
sessionid 0x138a0620045bbce, closing socket connection and attempting
reconnect
2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
Possibly transient ZooKeeper exception:
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO -
org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
Sleeping 4000ms before retry #2...
2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
Opening socket connection to server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
authenticate using SASL (unknown error)
2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
Socket connection established to
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
Session establishment complete on server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
0x138a0620045bbce, negotiated timeout = 180000
2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
Client session timed out, have not heard from server in 120001ms for
sessionid 0x138a0620045bbce, closing socket connection and attempting
reconnect
2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
Possibly transient ZooKeeper exception:
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO -
org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
Sleeping 8000ms before retry #3...
2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
Opening socket connection to server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
authenticate using SASL (unknown error)
2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
Socket connection established to
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
Session establishment complete on server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
0x138a0620045bbce, negotiated timeout = 180000
2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
Client session timed out, have not heard from server in 120000ms for
sessionid 0x138a0620045bbce, closing socket connection and attempting
reconnect
2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
Possibly transient ZooKeeper exception:
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR -
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)]
ZooKeeper exists failed after 3 retries
2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN -
org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)]
hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
Unable to set watcher on znode /hbase/master
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
        at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
        at
org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
        at
org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
        at
org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
        at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
        at
org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
        at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
        at
org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
        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$301(ScheduledThreadPoolExecutor.java:178)
        at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
        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:722)
2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR -
org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)]
hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
Received unexpected KeeperException, re-throwing exception
org.apache.zookeeper.KeeperException$ConnectionLossException:
KeeperErrorCode = ConnectionLoss for /hbase/master
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
        at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
        at
org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
        at
org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
        at
org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
        at
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
        at
org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
        at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
        at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
        at
org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
        at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
        at
org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
        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$301(ScheduledThreadPoolExecutor.java:178)
        at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
        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:722)
2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO -
org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)]
This client just lost it's session with ZooKeeper, will automatically
reconnect when needed.
2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
Opening socket connection to server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
authenticate using SASL (unknown error)
2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
Socket connection established to
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.com:2181)) [INFO -
org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
Session establishment complete on server
hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
0x138a0620045bbce, negotiated timeout = 180000

*zookeeper log*
2012-10-08 19:06:19,049 INFO
org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
connection from /10.10.10.64:16755
2012-10-08 19:06:19,053 INFO org.apache.zookeeper.server.ZooKeeperServer:
Client attempting to establish new session at /10.10.10.64:16755
2012-10-08 19:06:19,069 INFO org.apache.zookeeper.server.ZooKeeperServer:
Established session 0x138a0620045bbce with negotiated timeout 180000 for
client /10.10.10.64:16755
2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn:
Closed socket connection for client /10.10.10.64:16755 which had sessionid
0x138a0620045bbce
2012-10-08 19:08:20,692 INFO
org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
connection from /10.10.10.64:33245
2012-10-08 19:08:20,693 INFO org.apache.zookeeper.server.ZooKeeperServer:
Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:33245
2012-10-08 19:08:20,694 INFO org.apache.zookeeper.server.ZooKeeperServer:
Established session 0x138a0620045bbce with negotiated timeout 180000 for
client /10.10.10.64:33245
2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn:
caught end of stream exception
EndOfStreamException: Unable to read additional data from client sessionid
0x138a0620045bbce, likely client has closed socket
    at
org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
    at
org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
    at java.lang.Thread.run(Thread.java:619)
2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn:
Closed socket connection for client /10.10.10.64:33245 which had sessionid
0x138a0620045bbce
2012-10-08 19:10:22,794 INFO
org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
connection from /10.10.10.64:44993
2012-10-08 19:10:22,796 INFO org.apache.zookeeper.server.ZooKeeperServer:
Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:44993
2012-10-08 19:10:22,798 INFO org.apache.zookeeper.server.ZooKeeperServer:
Established session 0x138a0620045bbce with negotiated timeout 180000 for
client /10.10.10.64:44993


Thanks for the continued help from the community on getting us going w/flume

Cheers,

-- 
Iain Wright


This email message is confidential, intended only for the recipient(s)
named above and may contain information that is privileged, exempt from
disclosure under applicable law. If you are not the intended recipient, do
not disclose or disseminate the message to anyone except the intended
recipient. If you have received this message in error, or are not the named
recipient(s), please immediately notify the sender by return email, and
delete all copies of this message.

-- 
Iain Wright


<http://www.labctsi.org/>
This email message is confidential, intended only for the recipient(s)
named above and may contain information that is privileged, exempt from
disclosure under applicable law. If you are not the intended recipient, do
not disclose or disseminate the message to anyone except the intended
recipient. If you have received this message in error, or are not the named
recipient(s), please immediately notify the sender by return email, and
delete all copies of this message.

Re: Flume HbaseSink ZK woes

Posted by Hari Shreedharan <hs...@cloudera.com>.
Iain, 

Thanks for investigating. It looks like ZK 3.4.4 has the fix. So you should just be able to remove the older ZK version and drop the new one in.

Thanks,
Hari

-- 
Hari Shreedharan


On Monday, October 8, 2012 at 5:03 PM, iain wright wrote:

> - built latest 3.4 branch of zookeeper that included a commit/fix for the bug above: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/
> - ran 'ant' to build it
> - copied the new zookeeper jar to flume/lib directory
> - renamed the old one to flume-blah.old so the new one would be picked up
> 
> and we are now writing to hbase! 
> 
> Thanks,
> 
> -- 
> Iain Wright
> 
> 
 (http://www.labctsi.org/)> This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message.
> 
> 
> On Mon, Oct 8, 2012 at 4:23 PM, iain wright <iainwrig@gmail.com (mailto:iainwrig@gmail.com)> wrote:
> > Still no hits from the hbase guys,
> > 
> > I think we may getting bit by this:https://issues.apache.org/jira/browse/ZOOKEEPER-1437
> > 
> > Based on a ton of these in the debug console output: 
> > 
> > 2012-10-08 16:19:56,863 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.telescope.tv:2181 (http://hbasemaster0.hadoop.telescope.tv:2181))) [DEBUG - org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> > et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet: clientPath:null serverPath:null finished:false header:: 0,3  replyHeader:: 0,0,0  request:: '/hbase/ma
> > ster,T  response::  until SASL authentication completes.                                                                                                              
> > 2012-10-08 16:19:56,865 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.telescope.tv:2181 (http://hbasemaster0.hadoop.telescope.tv:2181))) [DEBUG - org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> > et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet: clientPath:null serverPath:null finished:false header:: 0,3  replyHeader:: 0,0,0  request:: '/hbase/ma
> > ster,T  response::  until SASL authentication completes.                                                                                                              
> > 2012-10-08 16:19:56,866 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.telescope.tv:2181 (http://hbasemaster0.hadoop.telescope.tv:2181))) [DEBUG - org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> > et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet: clientPath:null serverPath:null finished:false header:: -2,11  replyHeader:: null request:: null respo
> > nse:: nulluntil SASL authentication completes.                              
> > 
> > It looks like a patch is committed, but No idea how to upgrade zookeeper in the flume hbase client (or does it use a client from the hbase_home?) -- if that's even the root cause here: http://svn.apache.org/viewvc?view=revision&revision=1382555
> > 
> > 
> > Cheers,
> > -- 
> > Iain Wright
> > 
> > 
 (http://www.labctsi.org/)> > This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message.
> > 
> > 
> > On Mon, Oct 8, 2012 at 2:14 PM, iain wright <iainwrig@gmail.com (mailto:iainwrig@gmail.com)> wrote:
> > > Hi Hari,
> > > 
> > > Getting similar (but seemingly more serious/frequent) results with the ASync Sink, I'll hit the hbase list as well. Thank you 
> > > 
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:14,978 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:14,986 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd47, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd47 closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:19,108 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:19,110 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd48, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd48 closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:23,235 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:23,238 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd49, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd49 closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:27,368 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:27,373 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd4d, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd4d closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:31,500 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:31,504 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd4e, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd4e closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:35,660 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:35,668 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd4f, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd4f closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:39,815 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:39,818 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > :2181, initiating session                                                                                                            
> > > 11.30:2181, sessionid = 0x138a0620045bd50, negotiated timeout = 6000                                                                 
> > > on: 0x138a0620045bd50 closed                                                                                                         
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:43,947 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > 2012-10-08 14:09:43,952 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.tv/10.10.11.30 (http://hbasemaster0.hadoop.domain.tv/10.10.11.30)
> > > :2181, initiating session                                                                                                            
> > > 2012-10-08 14:09:43,968 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.tv/10.10 (http://hbasemaster0.hadoop.domain.tv/10.10).
> > > 11.30:2181, sessionid = 0x138a0620045bd51, negotiated timeout = 6000                                                                 
> > > 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO - org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:679)] Sessi
> > > on: 0x138a0620045bd51 closed                                                                                                         
> > > 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO - org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Init
> > > iating client connection, connectString=hbasemaster0.hadoop.domain.tv (http://hbasemaster0.hadoop.domain.tv) sessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> > > ient@66e43eb8                                                                                                                        
> > > 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [INFO - org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java
> > > :509)] EventThread shut down                                                                                                         
> > > 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [ERROR - org.hbase.async.HBaseClient$ZKClient$2.processResult(HBaseClie
> > > nt.java:2407)] Looks like our ZK session expired or is broken, rc=-4: CONNECTIONLOSS                                                 
> > > 2012-10-08 14:09:48,071 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > ntCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.tv/10.10.11 (http://hbasemaster0.hadoop.domain.tv/10.10.11)
> > > 
> > > .30:2181. Will not attempt to authenticate using SASL (unknown error)                                                                
> > > 2012-10-08 14:09:48,072 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.tv/10.10.11.30 (http://hbasemaster0.hadoop.domain.tv/10.10.11.30)
> > > :2181, initiating session                                                                                                            
> > > 2012-10-08 14:09:48,080 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.tv:2181 (http://hbasemaster0.hadoop.domain.tv:2181))) [INFO - org.apache.zookeeper.Clie
> > > ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.tv/10.10 (http://hbasemaster0.hadoop.domain.tv/10.10).
> > > 11.30:2181, sessionid = 0x138a0620045bd52, negotiated timeout = 6000            
> > > 
> > > -- 
> > > Iain Wright
> > > Cell: (562) 852-5916 (tel:%28562%29%20852-5916)
> > > 
 (http://www.labctsi.org/)> > > 
> > > This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message.
> > > 
> > > 
> > > On Mon, Oct 8, 2012 at 1:37 PM, Hari Shreedharan <hshreedharan@cloudera.com (mailto:hshreedharan@cloudera.com)> wrote:
> > > > Hi Iain,
> > > > 
> > > > I am not too sure of this issue. It looks like something to do with the HBaseClient. Can you try pinging HBase user list and see if this is a known issue? Did you try the async hbase sink? That is the recommended sink, I's suggest using that. 
> > > > 
> > > > Thanks,
> > > > Hari
> > > > 
> > > > 
> > > > -- 
> > > > Hari Shreedharan
> > > > 
> > > > 
> > > > On Monday, October 8, 2012 at 12:29 PM, iain wright wrote:
> > > > 
> > > > > We're having some trouble with the Flume & the HbaseSink. Seems we cannot hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo both look fine, Don't think we are hitting our hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions and I believe it defaults to 2k. I can establish new sessions using the CLI from the same server and idle there for 10 minutes without getting dropped. 
> > > > > 
> > > > > Flume & zookeeper logs below, using the same hadoop & hbase directories from our regionserver's.
> > > > > 
> > > > > Searched the list, this user appeared to have the same problem, not sure how he fixed it though:
> > > > > http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E
> > > > > 
> > > > > startup cmd
> > > > > /app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c ./conf -f conf/brian.properties  -Dflume.root.logger=INFO,console
> > > > > 
> > > > > flume-env.sh (http://flume-env.sh)
> > > > > $ cat conf/flume-env.sh (http://flume-env.sh) 
> > > > > 
> > > > > # Licensed to the Apache Software Foundation (ASF) under one
> > > > > # or more contributor license agreements.  See the NOTICE file
> > > > > # distributed with this work for additional information
> > > > > # regarding copyright ownership.  The ASF licenses this file
> > > > > # to you under the Apache License, Version 2.0 (the
> > > > > # "License"); you may not use this file except in compliance
> > > > > # with the License.  You may obtain a copy of the License at
> > > > > #
> > > > > #     http://www.apache.org/licenses/LICENSE-2.0
> > > > > #
> > > > > # Unless required by applicable law or agreed to in writing, software
> > > > > # distributed under the License is distributed on an "AS IS" BASIS,
> > > > > # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> > > > > # See the License for the specific language governing permissions and
> > > > > # limitations under the License.
> > > > > 
> > > > > # If this file is placed at FLUME_CONF_DIR/flume-env.sh (http://flume-env.sh), it will be sourced
> > > > > # during Flume startup.
> > > > > 
> > > > > # Enviroment variables can be set here.
> > > > > 
> > > > > #JAVA_HOME=/usr/lib/jvm/java-6-sun
> > > > > 
> > > > > # Give Flume more memory and pre-allocate, enable remote monitoring via JMX
> > > > > #JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"
> > > > > 
> > > > > # Note that the Flume conf directory is always included in the classpath.
> > > > > #FLUME_CLASSPATH="/app/flume/lib"
> > > > > 
> > > > > FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"
> > > > > 
> > > > > HBASE_HOME="/app/hbase-0.94.0"
> > > > > 
> > > > > HADOOP_HOME="/app/hadoop-1.0.1"
> > > > > 
> > > > > 
> > > > > config
> > > > > $ cat conf/brian.properties 
> > > > > #example.conf: A single-node Flume configuration
> > > > > 
> > > > > # Name the components on this agent
> > > > > agent1.sources = source1
> > > > > agent1.sinks = sink1
> > > > > agent1.channels = channel1
> > > > > 
> > > > > # Describe/configure source1
> > > > > agent1.sources.source1.type = exec 
> > > > > agent1.sources.source1.command = tail -F /tank/log_dev.log 
> > > > > agent1.sources.source1.batchSize = 1
> > > > > # Describe sink1
> > > > > #agent1.sinks.sink1.type = logger
> > > > > agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
> > > > > agent1.sinks.sink1.table = brian_test
> > > > > agent1.sinks.sink1.columnFamily = f1 
> > > > > #agent1.sinks.sink1.serializer = org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
> > > > > #agent1.sinks.sink1.serializer = org.apache.flume.sink.hbase.SimpleHbaseEventSerializer
> > > > > 
> > > > > 
> > > > > # Use a channel which buffers events in memory
> > > > > agent1.channels.channel1.type = memory
> > > > > agent1.channels.channel1.capacity = 1000
> > > > > agent1.channels.channel1.transactionCapactiy = 100
> > > > > 
> > > > > # Bind the source and sink to the channel
> > > > > agent1.sources.source1.channels = channel1
> > > > > agent1.sinks.sink1.channel = channel1
> > > > > 
> > > > > 
> > > > > flume console log
> > > > > 2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
> > > > > 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.io.tmpdir=/var/tmp/
> > > > > 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.compiler=<NA>
> > > > > 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.name (http://os.name)=FreeBSD
> > > > > 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.arch=amd64
> > > > > 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.version=9.0-RELEASE
> > > > > 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.name (http://user.name)=czhang
> > > > > 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.home=/users/czhang
> > > > > 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
> > > > > 2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating client connection, connectString=hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181) sessionTimeout=180000 watcher=hconnection
> > > > > 2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)] The identifier of this process is 12494@app14.app.domain.com (mailto:12494@app14.app.domain.com)
> > > > > 2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> > > > > 2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> > > > > 2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> > > > > 2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120001ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> > > > > 2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > > 2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 2000ms before retry #1...
> > > > > 2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> > > > > 2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> > > > > 2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> > > > > 2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120000ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> > > > > 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > > 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 4000ms before retry #2...
> > > > > 2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> > > > > 2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> > > > > 2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> > > > > 2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120001ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> > > > > 2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > > 2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 8000ms before retry #3...
> > > > > 2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> > > > > 2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> > > > > 2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> > > > > 2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120000ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> > > > > 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > > 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)] ZooKeeper exists failed after 3 retries
> > > > > 2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)] hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce Unable to set watcher on znode /hbase/master
> > > > > org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > >         at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
> > > > >         at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> > > > >         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
> > > > >         at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
> > > > >         at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
> > > > >         at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
> > > > >         at org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
> > > > >         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
> > > > >         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
> > > > >         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
> > > > >         at org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
> > > > >         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
> > > > >         at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
> > > > >         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$301(ScheduledThreadPoolExecutor.java:178)
> > > > >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> > > > >         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:722)
> > > > > 2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR - org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)] hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce Received unexpected KeeperException, re-throwing exception
> > > > > org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> > > > >         at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
> > > > >         at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
> > > > >         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
> > > > >         at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
> > > > >         at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
> > > > >         at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
> > > > >         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
> > > > >         at org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
> > > > >         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
> > > > >         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
> > > > >         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
> > > > >         at org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
> > > > >         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
> > > > >         at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
> > > > >         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$301(ScheduledThreadPoolExecutor.java:178)
> > > > >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> > > > >         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:722)
> > > > > 2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)] This client just lost it's session with ZooKeeper, will automatically reconnect when needed.
> > > > > 2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> > > > > 2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> > > > > 2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> > > > > 
> > > > > zookeeper log
> > > > > 2012-10-08 19:06:19,049 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:16755 (http://10.10.10.64:16755)
> > > > > 2012-10-08 19:06:19,053 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to establish new session at /10.10.10.64:16755 (http://10.10.10.64:16755)
> > > > > 2012-10-08 19:06:19,069 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:16755 (http://10.10.10.64:16755)
> > > > > 2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn: Closed socket connection for client /10.10.10.64:16755 (http://10.10.10.64:16755) which had sessionid 0x138a0620045bbce
> > > > > 2012-10-08 19:08:20,692 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:33245 (http://10.10.10.64:33245)
> > > > > 2012-10-08 19:08:20,693 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:33245 (http://10.10.10.64:33245)
> > > > > 2012-10-08 19:08:20,694 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:33245 (http://10.10.10.64:33245)
> > > > > 2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn: caught end of stream exception
> > > > > EndOfStreamException: Unable to read additional data from client sessionid 0x138a0620045bbce, likely client has closed socket
> > > > >     at org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
> > > > >     at org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
> > > > >     at java.lang.Thread.run(Thread.java:619)
> > > > > 2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn: Closed socket connection for client /10.10.10.64:33245 (http://10.10.10.64:33245) which had sessionid 0x138a0620045bbce
> > > > > 2012-10-08 19:10:22,794 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:44993 (http://10.10.10.64:44993)
> > > > > 2012-10-08 19:10:22,796 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:44993 (http://10.10.10.64:44993)
> > > > > 2012-10-08 19:10:22,798 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:44993 (http://10.10.10.64:44993)
> > > > > 
> > > > > 
> > > > > Thanks for the continued help from the community on getting us going w/flume
> > > > > 
> > > > > Cheers,
> > > > > 
> > > > > -- 
> > > > > Iain Wright
> > > > > 
> > > > > 
> > > > > This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message. 
> > > > > 
> > > > > -- 
> > > > > Iain Wright
> > > > > 
> > > > > 
 (http://www.labctsi.org/)> > > > > This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message.
> > > > 
> > > 
> > 
> 


Re: Flume HbaseSink ZK woes

Posted by iain wright <ia...@gmail.com>.
- built latest 3.4 branch of zookeeper that included a commit/fix for the
bug above: http://svn.apache.org/viewvc/zookeeper/branches/branch-3.4/
- ran 'ant' to build it
- copied the new zookeeper jar to flume/lib directory
- renamed the old one to flume-blah.old so the new one would be picked up

and we are now writing to hbase!

Thanks,

-- 
Iain Wright


<http://www.labctsi.org/>
This email message is confidential, intended only for the recipient(s)
named above and may contain information that is privileged, exempt from
disclosure under applicable law. If you are not the intended recipient, do
not disclose or disseminate the message to anyone except the intended
recipient. If you have received this message in error, or are not the named
recipient(s), please immediately notify the sender by return email, and
delete all copies of this message.


On Mon, Oct 8, 2012 at 4:23 PM, iain wright <ia...@gmail.com> wrote:

> Still no hits from the hbase guys,
>
> I think we may getting bit by this:
> https://issues.apache.org/jira/browse/ZOOKEEPER-1437
>
> Based on a ton of these in the debug console output:
>
> 2012-10-08 16:19:56,863 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
> org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
> clientPath:null serverPath:null finished:false header:: 0,3  replyHeader::
> 0,0,0  request:: '/hbase/ma
> ster,T  response::  until SASL authentication
> completes.
>
> 2012-10-08 16:19:56,865 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
> org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
> clientPath:null serverPath:null finished:false header:: 0,3  replyHeader::
> 0,0,0  request:: '/hbase/ma
> ster,T  response::  until SASL authentication
> completes.
>
> 2012-10-08 16:19:56,866 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
> org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
> et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
> clientPath:null serverPath:null finished:false header:: -2,11
> replyHeader:: null request:: null respo
> nse:: nulluntil SASL authentication
> completes.
>
> It looks like a patch is committed, but No idea how to upgrade zookeeper
> in the flume hbase client (or does it use a client from the hbase_home?) --
> if that's even the root cause here:
> http://svn.apache.org/viewvc?view=revision&revision=1382555
>
>
> Cheers,
> --
> Iain Wright
>
>
> <http://www.labctsi.org/>
> This email message is confidential, intended only for the recipient(s)
> named above and may contain information that is privileged, exempt from
> disclosure under applicable law. If you are not the intended recipient, do
> not disclose or disseminate the message to anyone except the intended
> recipient. If you have received this message in error, or are not the named
> recipient(s), please immediately notify the sender by return email, and
> delete all copies of this message.
>
>
> On Mon, Oct 8, 2012 at 2:14 PM, iain wright <ia...@gmail.com> wrote:
>
>> Hi Hari,
>>
>> Getting similar (but seemingly more serious/frequent) results with the
>> ASync Sink, I'll hit the hbase list as well. Thank you
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:14,978 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:14,986 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd47, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd47
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:19,108 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:19,110 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd48, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd48
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:23,235 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:23,238 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd49, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd49
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:27,368 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:27,373 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd4d, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd4d
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:31,500 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:31,504 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd4e, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd4e
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:35,660 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:35,668 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd4f, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd4f
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:39,815 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:39,818 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> :2181, initiating
>> session
>>
>> 11.30:2181, sessionid = 0x138a0620045bd50, negotiated timeout =
>> 6000
>> on: 0x138a0620045bd50
>> closed
>>
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:43,947 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> 2012-10-08 14:09:43,952 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
>> established to hbasemaster0.hadoop.domain.tv/10.10.11.30
>> :2181, initiating
>> session
>>
>> 2012-10-08 14:09:43,968 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session
>> establishment complete on server hbasemaster0.hadoop.domain.tv/10.10.
>> 11.30:2181, sessionid = 0x138a0620045bd51, negotiated timeout =
>> 6000
>> 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
>> org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:679)] Sessi
>> on: 0x138a0620045bd51
>> closed
>>
>> 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
>> org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Init
>> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
>> ient@66e43eb8
>>
>> 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [INFO -
>> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java
>> :509)] EventThread shut
>> down
>>
>> 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [ERROR -
>> org.hbase.async.HBaseClient$ZKClient$2.processResult(HBaseClie
>> nt.java:2407)] Looks like our ZK session expired or is broken, rc=-4:
>> CONNECTIONLOSS
>> 2012-10-08 14:09:48,071 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> ntCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket
>> connection to server hbasemaster0.hadoop.domain.tv/10.10.11
>>
>> .30:2181. Will not attempt to authenticate using SASL (unknown
>> error)
>> 2012-10-08 14:09:48,072 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
>> established to hbasemaster0.hadoop.domain.tv/10.10.11.30
>> :2181, initiating
>> session
>>
>> 2012-10-08 14:09:48,080 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
>> ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session
>> establishment complete on server hbasemaster0.hadoop.domain.tv/10.10.
>> 11.30:2181, sessionid = 0x138a0620045bd52, negotiated timeout =
>> 6000
>>
>> --
>> Iain Wright
>> Cell: (562) 852-5916
>>
>> <http://www.labctsi.org/>
>> This email message is confidential, intended only for the recipient(s)
>> named above and may contain information that is privileged, exempt from
>> disclosure under applicable law. If you are not the intended recipient, do
>> not disclose or disseminate the message to anyone except the intended
>> recipient. If you have received this message in error, or are not the named
>> recipient(s), please immediately notify the sender by return email, and
>> delete all copies of this message.
>>
>>
>> On Mon, Oct 8, 2012 at 1:37 PM, Hari Shreedharan <
>> hshreedharan@cloudera.com> wrote:
>>
>>> Hi Iain,
>>>
>>> I am not too sure of this issue. It looks like something to do with the
>>> HBaseClient. Can you try pinging HBase user list and see if this is a known
>>> issue? Did you try the async hbase sink? That is the recommended sink, I's
>>> suggest using that.
>>>
>>> Thanks,
>>> Hari
>>>
>>> --
>>> Hari Shreedharan
>>>
>>> On Monday, October 8, 2012 at 12:29 PM, iain wright wrote:
>>>
>>> We're having some trouble with the Flume & the HbaseSink. Seems we
>>> cannot hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo
>>> both look fine, Don't think we are hitting our
>>> hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions
>>> and I believe it defaults to 2k. I can establish new sessions using the CLI
>>> from the same server and idle there for 10 minutes without getting dropped.
>>>
>>> Flume & zookeeper logs below, using the same hadoop & hbase directories
>>> from our regionserver's.
>>>
>>> *Searched the list, this user appeared to have the same problem, not
>>> sure how he fixed it though:*
>>>
>>> http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E
>>>
>>> *startup cmd
>>> */app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c
>>> ./conf -f conf/brian.properties  -Dflume.root.logger=INFO,console*
>>> *
>>> *flume-env.sh*
>>> $ cat conf/flume-env.sh
>>>
>>> # Licensed to the Apache Software Foundation (ASF) under one
>>> # or more contributor license agreements.  See the NOTICE file
>>> # distributed with this work for additional information
>>> # regarding copyright ownership.  The ASF licenses this file
>>> # to you under the Apache License, Version 2.0 (the
>>> # "License"); you may not use this file except in compliance
>>> # with the License.  You may obtain a copy of the License at
>>> #
>>> #     http://www.apache.org/licenses/LICENSE-2.0
>>> #
>>> # Unless required by applicable law or agreed to in writing, software
>>> # distributed under the License is distributed on an "AS IS" BASIS,
>>> # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
>>> implied.
>>> # See the License for the specific language governing permissions and
>>> # limitations under the License.
>>>
>>> # If this file is placed at FLUME_CONF_DIR/flume-env.sh, it will be
>>> sourced
>>> # during Flume startup.
>>>
>>> # Enviroment variables can be set here.
>>>
>>> #JAVA_HOME=/usr/lib/jvm/java-6-sun
>>>
>>> # Give Flume more memory and pre-allocate, enable remote monitoring via
>>> JMX
>>> #JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"
>>>
>>> # Note that the Flume conf directory is always included in the classpath.
>>> #FLUME_CLASSPATH="/app/flume/lib"
>>>
>>> FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"
>>>
>>> HBASE_HOME="/app/hbase-0.94.0"
>>>
>>> HADOOP_HOME="/app/hadoop-1.0.1"
>>>
>>>
>>> *config*
>>> $ cat conf/brian.properties
>>> #example.conf: A single-node Flume configuration
>>>
>>> # Name the components on this agent
>>> agent1.sources = source1
>>> agent1.sinks = sink1
>>> agent1.channels = channel1
>>>
>>> # Describe/configure source1
>>> agent1.sources.source1.type = exec
>>> agent1.sources.source1.command = tail -F /tank/log_dev.log
>>> agent1.sources.source1.batchSize = 1
>>> # Describe sink1
>>> #agent1.sinks.sink1.type = logger
>>> agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
>>> agent1.sinks.sink1.table = brian_test
>>> agent1.sinks.sink1.columnFamily = f1
>>> #agent1.sinks.sink1.serializer =
>>> org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
>>> #agent1.sinks.sink1.serializer =
>>> org.apache.flume.sink.hbase.SimpleHbaseEventSerializer
>>>
>>>
>>> # Use a channel which buffers events in memory
>>> agent1.channels.channel1.type = memory
>>> agent1.channels.channel1.capacity = 1000
>>> agent1.channels.channel1.transactionCapactiy = 100
>>>
>>> # Bind the source and sink to the channel
>>> agent1.sources.source1.channels = channel1
>>> agent1.sinks.sink1.channel = channel1
>>>
>>>
>>> *flume console log*
>>> 2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
>>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:java.io.tmpdir=/var/tmp/
>>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:java.compiler=<NA>
>>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:os.name=FreeBSD
>>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:os.arch=amd64
>>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:os.version=9.0-RELEASE
>>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:user.name=czhang
>>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:user.home=/users/czhang
>>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>>> environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
>>> 2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating
>>> client connection, connectString=hbasemaster0.hadoop.domain.com:2181sessionTimeout=180000 watcher=hconnection
>>> 2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)]
>>> The identifier of this process is 12494@app14.app.domain.com
>>> 2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>>> Opening socket connection to server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>>> authenticate using SASL (unknown error)
>>> 2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>>> Socket connection established to
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>>> 2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>>> Session establishment complete on server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>>> 0x138a0620045bbce, negotiated timeout = 180000
>>> 2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>>> Client session timed out, have not heard from server in 120001ms for
>>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>>> reconnect
>>> 2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>>> Possibly transient ZooKeeper exception:
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>> 2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>>> Sleeping 2000ms before retry #1...
>>> 2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>>> Opening socket connection to server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>>> authenticate using SASL (unknown error)
>>> 2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>>> Socket connection established to
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>>> 2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>>> Session establishment complete on server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>>> 0x138a0620045bbce, negotiated timeout = 180000
>>> 2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>>> Client session timed out, have not heard from server in 120000ms for
>>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>>> reconnect
>>> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>>> Possibly transient ZooKeeper exception:
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>>> Sleeping 4000ms before retry #2...
>>> 2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>>> Opening socket connection to server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>>> authenticate using SASL (unknown error)
>>> 2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>>> Socket connection established to
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>>> 2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>>> Session establishment complete on server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>>> 0x138a0620045bbce, negotiated timeout = 180000
>>> 2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>>> Client session timed out, have not heard from server in 120001ms for
>>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>>> reconnect
>>> 2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>>> Possibly transient ZooKeeper exception:
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>> 2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>>> Sleeping 8000ms before retry #3...
>>> 2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>>> Opening socket connection to server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>>> authenticate using SASL (unknown error)
>>> 2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>>> Socket connection established to
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>>> 2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>>> Session establishment complete on server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>>> 0x138a0620045bbce, negotiated timeout = 180000
>>> 2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>>> Client session timed out, have not heard from server in 120000ms for
>>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>>> reconnect
>>> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>>> Possibly transient ZooKeeper exception:
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR -
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)]
>>> ZooKeeper exists failed after 3 retries
>>> 2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN -
>>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)]
>>> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
>>> Unable to set watcher on znode /hbase/master
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>>         at
>>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>>         at
>>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>>>         at
>>> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>>>         at
>>> org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>>>         at
>>> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>>>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>>>         at
>>> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>>>         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$301(ScheduledThreadPoolExecutor.java:178)
>>>         at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>>         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:722)
>>> 2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR -
>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)]
>>> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
>>> Received unexpected KeeperException, re-throwing exception
>>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>>         at
>>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>>         at
>>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>>>         at
>>> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>>>         at
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>>>         at
>>> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>>>         at
>>> org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>>>         at
>>> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>>>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>>>         at
>>> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>>>         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$301(ScheduledThreadPoolExecutor.java:178)
>>>         at
>>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>>         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:722)
>>> 2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO -
>>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)]
>>> This client just lost it's session with ZooKeeper, will automatically
>>> reconnect when needed.
>>> 2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>>> Opening socket connection to server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>>> authenticate using SASL (unknown error)
>>> 2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>>> Socket connection established to
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>>> 2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(
>>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>>> Session establishment complete on server
>>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>>> 0x138a0620045bbce, negotiated timeout = 180000
>>>
>>> *zookeeper log*
>>> 2012-10-08 19:06:19,049 INFO
>>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>>> connection from /10.10.10.64:16755
>>> 2012-10-08 19:06:19,053 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Client attempting to establish
>>> new session at /10.10.10.64:16755
>>> 2012-10-08 19:06:19,069 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Established session
>>> 0x138a0620045bbce with negotiated timeout 180000 for client /
>>> 10.10.10.64:16755
>>> 2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn:
>>> Closed socket connection for client /10.10.10.64:16755 which had
>>> sessionid 0x138a0620045bbce
>>> 2012-10-08 19:08:20,692 INFO
>>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>>> connection from /10.10.10.64:33245
>>> 2012-10-08 19:08:20,693 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew
>>> session 0x138a0620045bbce at /10.10.10.64:33245
>>> 2012-10-08 19:08:20,694 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Established session
>>> 0x138a0620045bbce with negotiated timeout 180000 for client /
>>> 10.10.10.64:33245
>>> 2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn:
>>> caught end of stream exception
>>> EndOfStreamException: Unable to read additional data from client
>>> sessionid 0x138a0620045bbce, likely client has closed socket
>>>     at
>>> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
>>>     at
>>> org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
>>>     at java.lang.Thread.run(Thread.java:619)
>>> 2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn:
>>> Closed socket connection for client /10.10.10.64:33245 which had
>>> sessionid 0x138a0620045bbce
>>> 2012-10-08 19:10:22,794 INFO
>>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>>> connection from /10.10.10.64:44993
>>> 2012-10-08 19:10:22,796 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew
>>> session 0x138a0620045bbce at /10.10.10.64:44993
>>> 2012-10-08 19:10:22,798 INFO
>>> org.apache.zookeeper.server.ZooKeeperServer: Established session
>>> 0x138a0620045bbce with negotiated timeout 180000 for client /
>>> 10.10.10.64:44993
>>>
>>>
>>> Thanks for the continued help from the community on getting us going
>>> w/flume
>>>
>>> Cheers,
>>>
>>> --
>>> Iain Wright
>>>
>>>
>>> This email message is confidential, intended only for the recipient(s)
>>> named above and may contain information that is privileged, exempt from
>>> disclosure under applicable law. If you are not the intended recipient, do
>>> not disclose or disseminate the message to anyone except the intended
>>> recipient. If you have received this message in error, or are not the named
>>> recipient(s), please immediately notify the sender by return email, and
>>> delete all copies of this message.
>>>
>>> --
>>> Iain Wright
>>>
>>>
>>> <http://www.labctsi.org/>
>>> This email message is confidential, intended only for the recipient(s)
>>> named above and may contain information that is privileged, exempt from
>>> disclosure under applicable law. If you are not the intended recipient, do
>>> not disclose or disseminate the message to anyone except the intended
>>> recipient. If you have received this message in error, or are not the named
>>> recipient(s), please immediately notify the sender by return email, and
>>> delete all copies of this message.
>>>
>>>
>>>
>>
>

Re: Flume HbaseSink ZK woes

Posted by iain wright <ia...@gmail.com>.
Still no hits from the hbase guys,

I think we may getting bit by this:
https://issues.apache.org/jira/browse/ZOOKEEPER-1437

Based on a ton of these in the debug console output:

2012-10-08 16:19:56,863 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
clientPath:null serverPath:null finished:false header:: 0,3  replyHeader::
0,0,0  request:: '/hbase/ma
ster,T  response::  until SASL authentication
completes.

2012-10-08 16:19:56,865 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
clientPath:null serverPath:null finished:false header:: 0,3  replyHeader::
0,0,0  request:: '/hbase/ma
ster,T  response::  until SASL authentication
completes.

2012-10-08 16:19:56,866 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.telescope.tv:2181)) [DEBUG -
org.apache.zookeeper.ClientCnxnSocketNIO.findSendablePack
et(ClientCnxnSocketNIO.java:164)] deferring non-priming packet:
clientPath:null serverPath:null finished:false header:: -2,11
replyHeader:: null request:: null respo
nse:: nulluntil SASL authentication completes.

It looks like a patch is committed, but No idea how to upgrade zookeeper in
the flume hbase client (or does it use a client from the hbase_home?) -- if
that's even the root cause here:
http://svn.apache.org/viewvc?view=revision&revision=1382555

Cheers,
-- 
Iain Wright


<http://www.labctsi.org/>
This email message is confidential, intended only for the recipient(s)
named above and may contain information that is privileged, exempt from
disclosure under applicable law. If you are not the intended recipient, do
not disclose or disseminate the message to anyone except the intended
recipient. If you have received this message in error, or are not the named
recipient(s), please immediately notify the sender by return email, and
delete all copies of this message.


On Mon, Oct 8, 2012 at 2:14 PM, iain wright <ia...@gmail.com> wrote:

> Hi Hari,
>
> Getting similar (but seemingly more serious/frequent) results with the
> ASync Sink, I'll hit the hbase list as well. Thank you
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:14,978 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:14,986 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd47, negotiated timeout =
> 6000
> on: 0x138a0620045bd47
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:19,108 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:19,110 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd48, negotiated timeout =
> 6000
> on: 0x138a0620045bd48
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:23,235 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:23,238 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd49, negotiated timeout =
> 6000
> on: 0x138a0620045bd49
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:27,368 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:27,373 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd4d, negotiated timeout =
> 6000
> on: 0x138a0620045bd4d
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:31,500 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:31,504 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd4e, negotiated timeout =
> 6000
> on: 0x138a0620045bd4e
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:35,660 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:35,668 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd4f, negotiated timeout =
> 6000
> on: 0x138a0620045bd4f
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:39,815 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:39,818 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> :2181, initiating
> session
>
> 11.30:2181, sessionid = 0x138a0620045bd50, negotiated timeout =
> 6000
> on: 0x138a0620045bd50
> closed
>
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:43,947 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> 2012-10-08 14:09:43,952 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
> established to hbasemaster0.hadoop.domain.tv/10.10.11.30
> :2181, initiating
> session
>
> 2012-10-08 14:09:43,968 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment
> complete on server hbasemaster0.hadoop.domain.tv/10.10.
> 11.30:2181, sessionid = 0x138a0620045bd51, negotiated timeout =
> 6000
> 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
> org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:679)] Sessi
> on: 0x138a0620045bd51
> closed
>
> 2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
> org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Init
> iating client connection, connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000 watcher=org.hbase.async.HBaseClient$ZKCl
> ient@66e43eb8
>
> 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [INFO -
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java
> :509)] EventThread shut
> down
>
> 2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [ERROR -
> org.hbase.async.HBaseClient$ZKClient$2.processResult(HBaseClie
> nt.java:2407)] Looks like our ZK session expired or is broken, rc=-4:
> CONNECTIONLOSS
> 2012-10-08 14:09:48,071 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> ntCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket
> connection to server hbasemaster0.hadoop.domain.tv/10.10.11
>
> .30:2181. Will not attempt to authenticate using SASL (unknown
> error)
> 2012-10-08 14:09:48,072 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
> established to hbasemaster0.hadoop.domain.tv/10.10.11.30
> :2181, initiating
> session
>
> 2012-10-08 14:09:48,080 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
> ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment
> complete on server hbasemaster0.hadoop.domain.tv/10.10.
> 11.30:2181, sessionid = 0x138a0620045bd52, negotiated timeout =
> 6000
>
> --
> Iain Wright
> Cell: (562) 852-5916
>
> <http://www.labctsi.org/>
> This email message is confidential, intended only for the recipient(s)
> named above and may contain information that is privileged, exempt from
> disclosure under applicable law. If you are not the intended recipient, do
> not disclose or disseminate the message to anyone except the intended
> recipient. If you have received this message in error, or are not the named
> recipient(s), please immediately notify the sender by return email, and
> delete all copies of this message.
>
>
> On Mon, Oct 8, 2012 at 1:37 PM, Hari Shreedharan <
> hshreedharan@cloudera.com> wrote:
>
>> Hi Iain,
>>
>> I am not too sure of this issue. It looks like something to do with the
>> HBaseClient. Can you try pinging HBase user list and see if this is a known
>> issue? Did you try the async hbase sink? That is the recommended sink, I's
>> suggest using that.
>>
>> Thanks,
>> Hari
>>
>> --
>> Hari Shreedharan
>>
>> On Monday, October 8, 2012 at 12:29 PM, iain wright wrote:
>>
>> We're having some trouble with the Flume & the HbaseSink. Seems we cannot
>> hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo both
>> look fine, Don't think we are hitting our
>> hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions
>> and I believe it defaults to 2k. I can establish new sessions using the CLI
>> from the same server and idle there for 10 minutes without getting dropped.
>>
>> Flume & zookeeper logs below, using the same hadoop & hbase directories
>> from our regionserver's.
>>
>> *Searched the list, this user appeared to have the same problem, not
>> sure how he fixed it though:*
>>
>> http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E
>>
>> *startup cmd
>> */app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c ./conf
>> -f conf/brian.properties  -Dflume.root.logger=INFO,console*
>> *
>> *flume-env.sh*
>> $ cat conf/flume-env.sh
>>
>> # Licensed to the Apache Software Foundation (ASF) under one
>> # or more contributor license agreements.  See the NOTICE file
>> # distributed with this work for additional information
>> # regarding copyright ownership.  The ASF licenses this file
>> # to you under the Apache License, Version 2.0 (the
>> # "License"); you may not use this file except in compliance
>> # with the License.  You may obtain a copy of the License at
>> #
>> #     http://www.apache.org/licenses/LICENSE-2.0
>> #
>> # Unless required by applicable law or agreed to in writing, software
>> # distributed under the License is distributed on an "AS IS" BASIS,
>> # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>> # See the License for the specific language governing permissions and
>> # limitations under the License.
>>
>> # If this file is placed at FLUME_CONF_DIR/flume-env.sh, it will be
>> sourced
>> # during Flume startup.
>>
>> # Enviroment variables can be set here.
>>
>> #JAVA_HOME=/usr/lib/jvm/java-6-sun
>>
>> # Give Flume more memory and pre-allocate, enable remote monitoring via
>> JMX
>> #JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"
>>
>> # Note that the Flume conf directory is always included in the classpath.
>> #FLUME_CLASSPATH="/app/flume/lib"
>>
>> FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"
>>
>> HBASE_HOME="/app/hbase-0.94.0"
>>
>> HADOOP_HOME="/app/hadoop-1.0.1"
>>
>>
>> *config*
>> $ cat conf/brian.properties
>> #example.conf: A single-node Flume configuration
>>
>> # Name the components on this agent
>> agent1.sources = source1
>> agent1.sinks = sink1
>> agent1.channels = channel1
>>
>> # Describe/configure source1
>> agent1.sources.source1.type = exec
>> agent1.sources.source1.command = tail -F /tank/log_dev.log
>> agent1.sources.source1.batchSize = 1
>> # Describe sink1
>> #agent1.sinks.sink1.type = logger
>> agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
>> agent1.sinks.sink1.table = brian_test
>> agent1.sinks.sink1.columnFamily = f1
>> #agent1.sinks.sink1.serializer =
>> org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
>> #agent1.sinks.sink1.serializer =
>> org.apache.flume.sink.hbase.SimpleHbaseEventSerializer
>>
>>
>> # Use a channel which buffers events in memory
>> agent1.channels.channel1.type = memory
>> agent1.channels.channel1.capacity = 1000
>> agent1.channels.channel1.transactionCapactiy = 100
>>
>> # Bind the source and sink to the channel
>> agent1.sources.source1.channels = channel1
>> agent1.sinks.sink1.channel = channel1
>>
>>
>> *flume console log*
>> 2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:java.io.tmpdir=/var/tmp/
>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:java.compiler=<NA>
>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:os.name=FreeBSD
>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:os.arch=amd64
>> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:os.version=9.0-RELEASE
>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:user.name=czhang
>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:user.home=/users/czhang
>> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
>> environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
>> 2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating
>> client connection, connectString=hbasemaster0.hadoop.domain.com:2181sessionTimeout=180000 watcher=hconnection
>> 2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)]
>> The identifier of this process is 12494@app14.app.domain.com
>> 2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>> Opening socket connection to server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>> authenticate using SASL (unknown error)
>> 2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>> Socket connection established to
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>> 2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>> Session establishment complete on server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>> 0x138a0620045bbce, negotiated timeout = 180000
>> 2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>> Client session timed out, have not heard from server in 120001ms for
>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>> reconnect
>> 2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>> Possibly transient ZooKeeper exception:
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>> 2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>> Sleeping 2000ms before retry #1...
>> 2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>> Opening socket connection to server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>> authenticate using SASL (unknown error)
>> 2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>> Socket connection established to
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>> 2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>> Session establishment complete on server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>> 0x138a0620045bbce, negotiated timeout = 180000
>> 2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>> Client session timed out, have not heard from server in 120000ms for
>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>> reconnect
>> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>> Possibly transient ZooKeeper exception:
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>> Sleeping 4000ms before retry #2...
>> 2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>> Opening socket connection to server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>> authenticate using SASL (unknown error)
>> 2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>> Socket connection established to
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>> 2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>> Session establishment complete on server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>> 0x138a0620045bbce, negotiated timeout = 180000
>> 2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>> Client session timed out, have not heard from server in 120001ms for
>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>> reconnect
>> 2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>> Possibly transient ZooKeeper exception:
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>> 2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
>> Sleeping 8000ms before retry #3...
>> 2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>> Opening socket connection to server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>> authenticate using SASL (unknown error)
>> 2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>> Socket connection established to
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>> 2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>> Session establishment complete on server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>> 0x138a0620045bbce, negotiated timeout = 180000
>> 2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
>> Client session timed out, have not heard from server in 120000ms for
>> sessionid 0x138a0620045bbce, closing socket connection and attempting
>> reconnect
>> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
>> Possibly transient ZooKeeper exception:
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR -
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)]
>> ZooKeeper exists failed after 3 retries
>> 2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN -
>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)]
>> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
>> Unable to set watcher on znode /hbase/master
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>         at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>         at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>>         at
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>>         at
>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>>         at
>> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>>         at
>> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>>         at
>> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>>         at
>> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>>         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$301(ScheduledThreadPoolExecutor.java:178)
>>         at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>         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:722)
>> 2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR -
>> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)]
>> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
>> Received unexpected KeeperException, re-throwing exception
>> org.apache.zookeeper.KeeperException$ConnectionLossException:
>> KeeperErrorCode = ConnectionLoss for /hbase/master
>>         at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>>         at
>> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>>         at
>> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>>         at
>> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>>         at
>> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>>         at
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>>         at
>> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>>         at
>> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>>         at
>> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>>         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$301(ScheduledThreadPoolExecutor.java:178)
>>         at
>> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>>         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:722)
>> 2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO -
>> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)]
>> This client just lost it's session with ZooKeeper, will automatically
>> reconnect when needed.
>> 2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
>> Opening socket connection to server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
>> authenticate using SASL (unknown error)
>> 2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
>> Socket connection established to
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
>> 2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(
>> hbasemaster0.hadoop.domain.com:2181)) [INFO -
>> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
>> Session establishment complete on server
>> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
>> 0x138a0620045bbce, negotiated timeout = 180000
>>
>> *zookeeper log*
>> 2012-10-08 19:06:19,049 INFO
>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>> connection from /10.10.10.64:16755
>> 2012-10-08 19:06:19,053 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Client attempting to establish new session at /10.10.10.64:16755
>> 2012-10-08 19:06:19,069 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Established session 0x138a0620045bbce with negotiated timeout 180000 for
>> client /10.10.10.64:16755
>> 2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn:
>> Closed socket connection for client /10.10.10.64:16755 which had
>> sessionid 0x138a0620045bbce
>> 2012-10-08 19:08:20,692 INFO
>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>> connection from /10.10.10.64:33245
>> 2012-10-08 19:08:20,693 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Client attempting to renew session 0x138a0620045bbce at /
>> 10.10.10.64:33245
>> 2012-10-08 19:08:20,694 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Established session 0x138a0620045bbce with negotiated timeout 180000 for
>> client /10.10.10.64:33245
>> 2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn:
>> caught end of stream exception
>> EndOfStreamException: Unable to read additional data from client
>> sessionid 0x138a0620045bbce, likely client has closed socket
>>     at
>> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
>>     at
>> org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
>>     at java.lang.Thread.run(Thread.java:619)
>> 2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn:
>> Closed socket connection for client /10.10.10.64:33245 which had
>> sessionid 0x138a0620045bbce
>> 2012-10-08 19:10:22,794 INFO
>> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
>> connection from /10.10.10.64:44993
>> 2012-10-08 19:10:22,796 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Client attempting to renew session 0x138a0620045bbce at /
>> 10.10.10.64:44993
>> 2012-10-08 19:10:22,798 INFO org.apache.zookeeper.server.ZooKeeperServer:
>> Established session 0x138a0620045bbce with negotiated timeout 180000 for
>> client /10.10.10.64:44993
>>
>>
>> Thanks for the continued help from the community on getting us going
>> w/flume
>>
>> Cheers,
>>
>> --
>> Iain Wright
>>
>>
>> This email message is confidential, intended only for the recipient(s)
>> named above and may contain information that is privileged, exempt from
>> disclosure under applicable law. If you are not the intended recipient, do
>> not disclose or disseminate the message to anyone except the intended
>> recipient. If you have received this message in error, or are not the named
>> recipient(s), please immediately notify the sender by return email, and
>> delete all copies of this message.
>>
>> --
>> Iain Wright
>>
>>
>> <http://www.labctsi.org/>
>> This email message is confidential, intended only for the recipient(s)
>> named above and may contain information that is privileged, exempt from
>> disclosure under applicable law. If you are not the intended recipient, do
>> not disclose or disseminate the message to anyone except the intended
>> recipient. If you have received this message in error, or are not the named
>> recipient(s), please immediately notify the sender by return email, and
>> delete all copies of this message.
>>
>>
>>
>

Re: Flume HbaseSink ZK woes

Posted by iain wright <ia...@gmail.com>.
Hi Hari,

Getting similar (but seemingly more serious/frequent) results with the
ASync Sink, I'll hit the hbase list as well. Thank you

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:14,978 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:14,986 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd47, negotiated timeout =
6000
on: 0x138a0620045bd47
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:19,108 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:19,110 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd48, negotiated timeout =
6000
on: 0x138a0620045bd48
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:23,235 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:23,238 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd49, negotiated timeout =
6000
on: 0x138a0620045bd49
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:27,368 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:27,373 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd4d, negotiated timeout =
6000
on: 0x138a0620045bd4d
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:31,500 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:31,504 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd4e, negotiated timeout =
6000
on: 0x138a0620045bd4e
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:35,660 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:35,668 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd4f, negotiated timeout =
6000
on: 0x138a0620045bd4f
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:39,815 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:39,818 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
:2181, initiating
session

11.30:2181, sessionid = 0x138a0620045bd50, negotiated timeout =
6000
on: 0x138a0620045bd50
closed

iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
:509)] EventThread shut
down

2012-10-08 14:09:43,947 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
2012-10-08 14:09:43,952 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
established to hbasemaster0.hadoop.domain.tv/10.10.11.30
:2181, initiating
session

2012-10-08 14:09:43,968 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment
complete on server hbasemaster0.hadoop.domain.tv/10.10.
11.30:2181, sessionid = 0x138a0620045bd51, negotiated timeout =
6000
2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:679)] Sessi
on: 0x138a0620045bd51
closed

2012-10-08 14:09:48,069 (lifecycleSupervisor-1-1-EventThread) [INFO -
org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Init
iating client connection,
connectString=hbasemaster0.hadoop.domain.tvsessionTimeout=5000
watcher=org.hbase.async.HBaseClient$ZKCl
ient@66e43eb8

2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [INFO -
org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java
:509)] EventThread shut
down

2012-10-08 14:09:48,070 (lifecycleSupervisor-1-1-EventThread) [ERROR -
org.hbase.async.HBaseClient$ZKClient$2.processResult(HBaseClie
nt.java:2407)] Looks like our ZK session expired or is broken, rc=-4:
CONNECTIONLOSS
2012-10-08 14:09:48,071 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
ntCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket
connection to server hbasemaster0.hadoop.domain.tv/10.10.11
.30:2181. Will not attempt to authenticate using SASL (unknown
error)
2012-10-08 14:09:48,072 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
ntCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection
established to hbasemaster0.hadoop.domain.tv/10.10.11.30
:2181, initiating
session

2012-10-08 14:09:48,080 (lifecycleSupervisor-1-1-SendThread(
hbasemaster0.hadoop.domain.tv:2181)) [INFO - org.apache.zookeeper.Clie
ntCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment
complete on server hbasemaster0.hadoop.domain.tv/10.10.
11.30:2181, sessionid = 0x138a0620045bd52, negotiated timeout =
6000

-- 
Iain Wright
Cell: (562) 852-5916

<http://www.labctsi.org/>
This email message is confidential, intended only for the recipient(s)
named above and may contain information that is privileged, exempt from
disclosure under applicable law. If you are not the intended recipient, do
not disclose or disseminate the message to anyone except the intended
recipient. If you have received this message in error, or are not the named
recipient(s), please immediately notify the sender by return email, and
delete all copies of this message.


On Mon, Oct 8, 2012 at 1:37 PM, Hari Shreedharan
<hs...@cloudera.com>wrote:

> Hi Iain,
>
> I am not too sure of this issue. It looks like something to do with the
> HBaseClient. Can you try pinging HBase user list and see if this is a known
> issue? Did you try the async hbase sink? That is the recommended sink, I's
> suggest using that.
>
> Thanks,
> Hari
>
> --
> Hari Shreedharan
>
> On Monday, October 8, 2012 at 12:29 PM, iain wright wrote:
>
> We're having some trouble with the Flume & the HbaseSink. Seems we cannot
> hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo both
> look fine, Don't think we are hitting our
> hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions
> and I believe it defaults to 2k. I can establish new sessions using the CLI
> from the same server and idle there for 10 minutes without getting dropped.
>
> Flume & zookeeper logs below, using the same hadoop & hbase directories
> from our regionserver's.
>
> *Searched the list, this user appeared to have the same problem, not sure
> how he fixed it though:*
>
> http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E
>
> *startup cmd
> */app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c ./conf
> -f conf/brian.properties  -Dflume.root.logger=INFO,console*
> *
> *flume-env.sh*
> $ cat conf/flume-env.sh
>
> # Licensed to the Apache Software Foundation (ASF) under one
> # or more contributor license agreements.  See the NOTICE file
> # distributed with this work for additional information
> # regarding copyright ownership.  The ASF licenses this file
> # to you under the Apache License, Version 2.0 (the
> # "License"); you may not use this file except in compliance
> # with the License.  You may obtain a copy of the License at
> #
> #     http://www.apache.org/licenses/LICENSE-2.0
> #
> # Unless required by applicable law or agreed to in writing, software
> # distributed under the License is distributed on an "AS IS" BASIS,
> # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> # See the License for the specific language governing permissions and
> # limitations under the License.
>
> # If this file is placed at FLUME_CONF_DIR/flume-env.sh, it will be
> sourced
> # during Flume startup.
>
> # Enviroment variables can be set here.
>
> #JAVA_HOME=/usr/lib/jvm/java-6-sun
>
> # Give Flume more memory and pre-allocate, enable remote monitoring via JMX
> #JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"
>
> # Note that the Flume conf directory is always included in the classpath.
> #FLUME_CLASSPATH="/app/flume/lib"
>
> FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"
>
> HBASE_HOME="/app/hbase-0.94.0"
>
> HADOOP_HOME="/app/hadoop-1.0.1"
>
>
> *config*
> $ cat conf/brian.properties
> #example.conf: A single-node Flume configuration
>
> # Name the components on this agent
> agent1.sources = source1
> agent1.sinks = sink1
> agent1.channels = channel1
>
> # Describe/configure source1
> agent1.sources.source1.type = exec
> agent1.sources.source1.command = tail -F /tank/log_dev.log
> agent1.sources.source1.batchSize = 1
> # Describe sink1
> #agent1.sinks.sink1.type = logger
> agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
> agent1.sinks.sink1.table = brian_test
> agent1.sinks.sink1.columnFamily = f1
> #agent1.sinks.sink1.serializer =
> org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
> #agent1.sinks.sink1.serializer =
> org.apache.flume.sink.hbase.SimpleHbaseEventSerializer
>
>
> # Use a channel which buffers events in memory
> agent1.channels.channel1.type = memory
> agent1.channels.channel1.capacity = 1000
> agent1.channels.channel1.transactionCapactiy = 100
>
> # Bind the source and sink to the channel
> agent1.sources.source1.channels = channel1
> agent1.sinks.sink1.channel = channel1
>
>
> *flume console log*
> 2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:java.io.tmpdir=/var/tmp/
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:java.compiler=<NA>
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:os.name=FreeBSD
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:os.arch=amd64
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:os.version=9.0-RELEASE
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:user.name=czhang
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:user.home=/users/czhang
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client
> environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
> 2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO -
> org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating
> client connection, connectString=hbasemaster0.hadoop.domain.com:2181sessionTimeout=180000 watcher=hconnection
> 2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)]
> The identifier of this process is 12494@app14.app.domain.com
> 2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
> Opening socket connection to server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
> authenticate using SASL (unknown error)
> 2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
> Socket connection established to
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
> 2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
> Session establishment complete on server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
> 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
> Client session timed out, have not heard from server in 120001ms for
> sessionid 0x138a0620045bbce, closing socket connection and attempting
> reconnect
> 2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
> Possibly transient ZooKeeper exception:
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO -
> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
> Sleeping 2000ms before retry #1...
> 2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
> Opening socket connection to server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
> authenticate using SASL (unknown error)
> 2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
> Socket connection established to
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
> 2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
> Session establishment complete on server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
> 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
> Client session timed out, have not heard from server in 120000ms for
> sessionid 0x138a0620045bbce, closing socket connection and attempting
> reconnect
> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
> Possibly transient ZooKeeper exception:
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO -
> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
> Sleeping 4000ms before retry #2...
> 2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
> Opening socket connection to server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
> authenticate using SASL (unknown error)
> 2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
> Socket connection established to
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
> 2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
> Session establishment complete on server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
> 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
> Client session timed out, have not heard from server in 120001ms for
> sessionid 0x138a0620045bbce, closing socket connection and attempting
> reconnect
> 2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
> Possibly transient ZooKeeper exception:
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO -
> org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)]
> Sleeping 8000ms before retry #3...
> 2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
> Opening socket connection to server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
> authenticate using SASL (unknown error)
> 2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
> Socket connection established to
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
> 2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
> Session establishment complete on server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
> 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)]
> Client session timed out, have not heard from server in 120000ms for
> sessionid 0x138a0620045bbce, closing socket connection and attempting
> reconnect
> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)]
> Possibly transient ZooKeeper exception:
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR -
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)]
> ZooKeeper exists failed after 3 retries
> 2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN -
> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)]
> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
> Unable to set watcher on znode /hbase/master
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
>         at
> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>         at
> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>         at
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>         at
> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>         at
> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>         at
> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>         at
> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>         at
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>         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$301(ScheduledThreadPoolExecutor.java:178)
>         at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         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:722)
> 2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR -
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)]
> hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce
> Received unexpected KeeperException, re-throwing exception
> org.apache.zookeeper.KeeperException$ConnectionLossException:
> KeeperErrorCode = ConnectionLoss for /hbase/master
>         at
> org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>         at
> org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>         at
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>         at
> org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>         at
> org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>         at
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>         at
> org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>         at
> org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>         at
> org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>         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$301(ScheduledThreadPoolExecutor.java:178)
>         at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         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:722)
> 2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO -
> org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)]
> This client just lost it's session with ZooKeeper, will automatically
> reconnect when needed.
> 2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)]
> Opening socket connection to server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181. Will not attempt to
> authenticate using SASL (unknown error)
> 2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)]
> Socket connection established to
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, initiating session
> 2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(
> hbasemaster0.hadoop.domain.com:2181)) [INFO -
> org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)]
> Session establishment complete on server
> hbasemaster0.hadoop.domain.com/10.10.11.30:2181, sessionid =
> 0x138a0620045bbce, negotiated timeout = 180000
>
> *zookeeper log*
> 2012-10-08 19:06:19,049 INFO
> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
> connection from /10.10.10.64:16755
> 2012-10-08 19:06:19,053 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Client attempting to establish new session at /10.10.10.64:16755
> 2012-10-08 19:06:19,069 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Established session 0x138a0620045bbce with negotiated timeout 180000 for
> client /10.10.10.64:16755
> 2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn:
> Closed socket connection for client /10.10.10.64:16755 which had
> sessionid 0x138a0620045bbce
> 2012-10-08 19:08:20,692 INFO
> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
> connection from /10.10.10.64:33245
> 2012-10-08 19:08:20,693 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:33245
> 2012-10-08 19:08:20,694 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Established session 0x138a0620045bbce with negotiated timeout 180000 for
> client /10.10.10.64:33245
> 2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn:
> caught end of stream exception
> EndOfStreamException: Unable to read additional data from client sessionid
> 0x138a0620045bbce, likely client has closed socket
>     at
> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
>     at
> org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
>     at java.lang.Thread.run(Thread.java:619)
> 2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn:
> Closed socket connection for client /10.10.10.64:33245 which had
> sessionid 0x138a0620045bbce
> 2012-10-08 19:10:22,794 INFO
> org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket
> connection from /10.10.10.64:44993
> 2012-10-08 19:10:22,796 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:44993
> 2012-10-08 19:10:22,798 INFO org.apache.zookeeper.server.ZooKeeperServer:
> Established session 0x138a0620045bbce with negotiated timeout 180000 for
> client /10.10.10.64:44993
>
>
> Thanks for the continued help from the community on getting us going
> w/flume
>
> Cheers,
>
> --
> Iain Wright
>
>
> This email message is confidential, intended only for the recipient(s)
> named above and may contain information that is privileged, exempt from
> disclosure under applicable law. If you are not the intended recipient, do
> not disclose or disseminate the message to anyone except the intended
> recipient. If you have received this message in error, or are not the named
> recipient(s), please immediately notify the sender by return email, and
> delete all copies of this message.
>
> --
> Iain Wright
>
>
> <http://www.labctsi.org/>
> This email message is confidential, intended only for the recipient(s)
> named above and may contain information that is privileged, exempt from
> disclosure under applicable law. If you are not the intended recipient, do
> not disclose or disseminate the message to anyone except the intended
> recipient. If you have received this message in error, or are not the named
> recipient(s), please immediately notify the sender by return email, and
> delete all copies of this message.
>
>
>

Re: Flume HbaseSink ZK woes

Posted by Hari Shreedharan <hs...@cloudera.com>.
Hi Iain,

I am not too sure of this issue. It looks like something to do with the HBaseClient. Can you try pinging HBase user list and see if this is a known issue? Did you try the async hbase sink? That is the recommended sink, I's suggest using that.

Thanks,
Hari


-- 
Hari Shreedharan


On Monday, October 8, 2012 at 12:29 PM, iain wright wrote:

> We're having some trouble with the Flume & the HbaseSink. Seems we cannot hang on to zookeeper sessions. Using 1.3 ng, hbase 0.94. Hbase & Zoo both look fine, Don't think we are hitting our hbase.zookeeper.property.maxClientCnxns as we only have about 95 sesions and I believe it defaults to 2k. I can establish new sessions using the CLI from the same server and idle there for 10 minutes without getting dropped. 
> 
> Flume & zookeeper logs below, using the same hadoop & hbase directories from our regionserver's.
> 
> Searched the list, this user appeared to have the same problem, not sure how he fixed it though:
> http://mail-archives.apache.org/mod_mbox/flume-user/201207.mbox/%3CCADGpRhcgztwhuUJi1izMRX2UOKeAWwhYOkKSKkHYmKopAkwHAQ@mail.gmail.com%3E
> 
> startup cmd
> /app/apache-flume-1.3.0-SNAPSHOT/bin/flume-ng agent -n agent1 -c ./conf -f conf/brian.properties  -Dflume.root.logger=INFO,console
> 
> flume-env.sh (http://flume-env.sh)
> $ cat conf/flume-env.sh (http://flume-env.sh) 
> 
> # Licensed to the Apache Software Foundation (ASF) under one
> # or more contributor license agreements.  See the NOTICE file
> # distributed with this work for additional information
> # regarding copyright ownership.  The ASF licenses this file
> # to you under the Apache License, Version 2.0 (the
> # "License"); you may not use this file except in compliance
> # with the License.  You may obtain a copy of the License at
> #
> #     http://www.apache.org/licenses/LICENSE-2.0
> #
> # Unless required by applicable law or agreed to in writing, software
> # distributed under the License is distributed on an "AS IS" BASIS,
> # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> # See the License for the specific language governing permissions and
> # limitations under the License.
> 
> # If this file is placed at FLUME_CONF_DIR/flume-env.sh (http://flume-env.sh), it will be sourced
> # during Flume startup.
> 
> # Enviroment variables can be set here.
> 
> #JAVA_HOME=/usr/lib/jvm/java-6-sun
> 
> # Give Flume more memory and pre-allocate, enable remote monitoring via JMX
> #JAVA_OPTS="-Xms100m -Xmx200m -Dcom.sun.management.jmxremote"
> 
> # Note that the Flume conf directory is always included in the classpath.
> #FLUME_CLASSPATH="/app/flume/lib"
> 
> FLUME_CLASSPATH="/app/apache-flume-1.3.0-SNAPSHOT/lib"
> 
> HBASE_HOME="/app/hbase-0.94.0"
> 
> HADOOP_HOME="/app/hadoop-1.0.1"
> 
> 
> config
> $ cat conf/brian.properties 
> #example.conf: A single-node Flume configuration
> 
> # Name the components on this agent
> agent1.sources = source1
> agent1.sinks = sink1
> agent1.channels = channel1
> 
> # Describe/configure source1
> agent1.sources.source1.type = exec 
> agent1.sources.source1.command = tail -F /tank/log_dev.log 
> agent1.sources.source1.batchSize = 1
> # Describe sink1
> #agent1.sinks.sink1.type = logger
> agent1.sinks.sink1.type = org.apache.flume.sink.hbase.HBaseSink
> agent1.sinks.sink1.table = brian_test
> agent1.sinks.sink1.columnFamily = f1 
> #agent1.sinks.sink1.serializer = org.apache.flume.sink.hbase.SimpleHBaseEventSerializer
> #agent1.sinks.sink1.serializer = org.apache.flume.sink.hbase.SimpleHbaseEventSerializer
> 
> 
> # Use a channel which buffers events in memory
> agent1.channels.channel1.type = memory
> agent1.channels.channel1.capacity = 1000
> agent1.channels.channel1.transactionCapactiy = 100
> 
> # Bind the source and sink to the channel
> agent1.sources.source1.channels = channel1
> agent1.sinks.sink1.channel = channel1
> 
> 
> flume console log
> 2012-10-08 12:06:19,007 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.library.path=:/app/hadoop-1.0.1/libexec/../lib/native/FreeBSD-amd64-64:/app/hbase-0.94.0/bin/../lib/native/FreeBSD-amd64-64
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.io.tmpdir=/var/tmp/
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:java.compiler=<NA>
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.name (http://os.name)=FreeBSD
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.arch=amd64
> 2012-10-08 12:06:19,008 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:os.version=9.0-RELEASE
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.name (http://user.name)=czhang
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.home=/users/czhang
> 2012-10-08 12:06:19,009 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.Environment.logEnv(Environment.java:100)] Client environment:user.dir=/app/apache-flume-1.3.0-SNAPSHOT
> 2012-10-08 12:06:19,010 (lifecycleSupervisor-1-1) [INFO - org.apache.zookeeper.ZooKeeper.<init>(ZooKeeper.java:433)] Initiating client connection, connectString=hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181) sessionTimeout=180000 watcher=hconnection
> 2012-10-08 12:06:19,036 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.<init>(RecoverableZooKeeper.java:97)] The identifier of this process is 12494@app14.app.domain.com (mailto:12494@app14.app.domain.com)
> 2012-10-08 12:06:19,041 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> 2012-10-08 12:06:19,049 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> 2012-10-08 12:06:19,072 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:08:19,070 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120001ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> 2012-10-08 12:08:19,233 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:08:19,234 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 2000ms before retry #1...
> 2012-10-08 12:08:20,691 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> 2012-10-08 12:08:20,692 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> 2012-10-08 12:08:20,695 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:10:20,695 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120000ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:10:20,796 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 4000ms before retry #2...
> 2012-10-08 12:10:22,792 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> 2012-10-08 12:10:22,794 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> 2012-10-08 12:10:22,799 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:12:22,800 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120001ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> 2012-10-08 12:12:22,902 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:12:22,903 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.util.RetryCounter.sleepUntilNextRetry(RetryCounter.java:53)] Sleeping 8000ms before retry #3...
> 2012-10-08 12:12:24,289 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> 2012-10-08 12:12:24,291 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> 2012-10-08 12:12:24,294 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> 2012-10-08 12:14:24,294 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1083)] Client session timed out, have not heard from server in 120000ms for sessionid 0x138a0620045bbce, closing socket connection and attempting reconnect
> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:195)] Possibly transient ZooKeeper exception: org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
> 2012-10-08 12:14:24,395 (lifecycleSupervisor-1-1) [ERROR - org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.retryOrThrow(RecoverableZooKeeper.java:197)] ZooKeeper exists failed after 3 retries
> 2012-10-08 12:14:24,398 (lifecycleSupervisor-1-1) [WARN - org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:239)] hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce Unable to set watcher on znode /hbase/master
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>         at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>         at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>         at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>         at org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>         at org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>         at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>         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$301(ScheduledThreadPoolExecutor.java:178)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         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:722)
> 2012-10-08 12:14:24,405 (lifecycleSupervisor-1-1) [ERROR - org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.keeperException(ZooKeeperWatcher.java:408)] hconnection-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce-0x138a0620045bbce Received unexpected KeeperException, re-throwing exception
> org.apache.zookeeper.KeeperException$ConnectionLossException: KeeperErrorCode = ConnectionLoss for /hbase/master
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:99)
>         at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
>         at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1036)
>         at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.exists(RecoverableZooKeeper.java:150)
>         at org.apache.hadoop.hbase.zookeeper.ZKUtil.watchAndCheckExists(ZKUtil.java:230)
>         at org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker.start(ZooKeeperNodeTracker.java:82)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.ensureZookeeperTrackers(HConnectionManager.java:590)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:818)
>         at org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.locateRegion(HConnectionManager.java:801)
>         at org.apache.hadoop.hbase.client.HTable.finishSetup(HTable.java:234)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:174)
>         at org.apache.hadoop.hbase.client.HTable.<init>(HTable.java:133)
>         at org.apache.flume.sink.hbase.HBaseSink.start(HBaseSink.java:107)
>         at org.apache.flume.sink.DefaultSinkProcessor.start(DefaultSinkProcessor.java:46)
>         at org.apache.flume.SinkRunner.start(SinkRunner.java:79)
>         at org.apache.flume.lifecycle.LifecycleSupervisor$MonitorRunnable.run(LifecycleSupervisor.java:236)
>         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$301(ScheduledThreadPoolExecutor.java:178)
>         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
>         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:722)
> 2012-10-08 12:14:24,407 (lifecycleSupervisor-1-1) [INFO - org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation.abort(HConnectionManager.java:1654)] This client just lost it's session with ZooKeeper, will automatically reconnect when needed.
> 2012-10-08 12:14:26,166 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.logStartConnect(ClientCnxn.java:966)] Opening socket connection to server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181). Will not attempt to authenticate using SASL (unknown error)
> 2012-10-08 12:14:26,167 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.primeConnection(ClientCnxn.java:849)] Socket connection established to hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), initiating session
> 2012-10-08 12:14:26,170 (lifecycleSupervisor-1-1-SendThread(hbasemaster0.hadoop.domain.com:2181 (http://hbasemaster0.hadoop.domain.com:2181))) [INFO - org.apache.zookeeper.ClientCnxn$SendThread.onConnected(ClientCnxn.java:1207)] Session establishment complete on server hbasemaster0.hadoop.domain.com/10.10.11.30:2181 (http://hbasemaster0.hadoop.domain.com/10.10.11.30:2181), sessionid = 0x138a0620045bbce, negotiated timeout = 180000
> 
> zookeeper log
> 2012-10-08 19:06:19,049 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:16755 (http://10.10.10.64:16755)
> 2012-10-08 19:06:19,053 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to establish new session at /10.10.10.64:16755 (http://10.10.10.64:16755)
> 2012-10-08 19:06:19,069 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:16755 (http://10.10.10.64:16755)
> 2012-10-08 19:08:19,072 INFO org.apache.zookeeper.server.NIOServerCnxn: Closed socket connection for client /10.10.10.64:16755 (http://10.10.10.64:16755) which had sessionid 0x138a0620045bbce
> 2012-10-08 19:08:20,692 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:33245 (http://10.10.10.64:33245)
> 2012-10-08 19:08:20,693 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:33245 (http://10.10.10.64:33245)
> 2012-10-08 19:08:20,694 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:33245 (http://10.10.10.64:33245)
> 2012-10-08 19:10:20,695 WARN org.apache.zookeeper.server.NIOServerCnxn: caught end of stream exception
> EndOfStreamException: Unable to read additional data from client sessionid 0x138a0620045bbce, likely client has closed socket
>     at org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:220)
>     at org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:224)
>     at java.lang.Thread.run(Thread.java:619)
> 2012-10-08 19:10:20,698 INFO org.apache.zookeeper.server.NIOServerCnxn: Closed socket connection for client /10.10.10.64:33245 (http://10.10.10.64:33245) which had sessionid 0x138a0620045bbce
> 2012-10-08 19:10:22,794 INFO org.apache.zookeeper.server.NIOServerCnxnFactory: Accepted socket connection from /10.10.10.64:44993 (http://10.10.10.64:44993)
> 2012-10-08 19:10:22,796 INFO org.apache.zookeeper.server.ZooKeeperServer: Client attempting to renew session 0x138a0620045bbce at /10.10.10.64:44993 (http://10.10.10.64:44993)
> 2012-10-08 19:10:22,798 INFO org.apache.zookeeper.server.ZooKeeperServer: Established session 0x138a0620045bbce with negotiated timeout 180000 for client /10.10.10.64:44993 (http://10.10.10.64:44993)
> 
> 
> Thanks for the continued help from the community on getting us going w/flume
> 
> Cheers,
> 
> -- 
> Iain Wright
> 
> 
> This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message. 
> 
> -- 
> Iain Wright
> 
> 
 (http://www.labctsi.org/)> This email message is confidential, intended only for the recipient(s) named above and may contain information that is privileged, exempt from disclosure under applicable law. If you are not the intended recipient, do not disclose or disseminate the message to anyone except the intended recipient. If you have received this message in error, or are not the named recipient(s), please immediately notify the sender by return email, and delete all copies of this message.