You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-user@hadoop.apache.org by Davey Yan <da...@gmail.com> on 2013/02/25 10:10:33 UTC

Datanodes shutdown and HBase's regionservers not working

Hey guys,

We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
year, and it works fine.
But the datanodes got shutdown twice in the last month.

When the datanodes got shutdown, all of them became "Dead Nodes" in
the NN web admin UI(http://ip:50070/dfshealth.jsp),
but regionservers of HBase were still live in the HBase web
admin(http://ip:60010/master-status), of course, they were zombies.
All of the processes of jvm were still running, including
hmaster/namenode/regionserver/datanode.

When the datanodes got shutdown, the load (using the "top" command) of
slaves became very high, more than 10, higher than normal running.
>From the "top" command, we saw that the processes of datanode and
regionserver were comsuming CPU.

We could not stop the HBase or Hadoop cluster through normal
commands(stop-*.sh/*-daemon.sh stop *).
So we stopped datanodes and regionservers by kill -9 PID, then the
load of slaves returned to normal level, and we start the cluster
again.


Log of NN at the shutdown point(All of the DNs were removed):
2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
Removing a node: /default-rack/192.168.1.152:50010
2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
192.168.1.149:50010
2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
Removing a node: /default-rack/192.168.1.149:50010
2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
192.168.1.150:50010
2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
Removing a node: /default-rack/192.168.1.150:50010
2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
192.168.1.148:50010
2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
Removing a node: /default-rack/192.168.1.148:50010


Logs in DNs indicated there were many IOException and SocketTimeoutException:
2013-02-22 11:02:52,354 ERROR
org.apache.hadoop.hdfs.server.datanode.DataNode:
DatanodeRegistration(192.168.1.148:50010,
storageID=DS-970284113-117.25.149.160-50010-1328074119937,
infoPort=50075, ipcPort=50020):DataXceiver
java.io.IOException: Interrupted receiveBlock
	at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
	at java.lang.Thread.run(Thread.java:662)
2013-02-22 11:03:44,823 WARN
org.apache.hadoop.hdfs.server.datanode.DataNode:
DatanodeRegistration(192.168.1.148:50010,
storageID=DS-970284113-117.25.149.160-50010-1328074119937,
infoPort=50075, ipcPort=50020):Got exception while serving
blk_-1985405101514576650_247001 to /192.168.1.148:
java.net.SocketTimeoutException: 480000 millis timeout while waiting
for channel to be ready for write. ch :
java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
remote=/192.168.1.148:48654]
	at org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
	at org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
	at org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
	at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
	at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
	at java.lang.Thread.run(Thread.java:662)
2013-02-22 11:09:42,294 ERROR
org.apache.hadoop.hdfs.server.datanode.DataNode:
DatanodeRegistration(192.168.1.148:50010,
storageID=DS-970284113-117.25.149.160-50010-1328074119937,
infoPort=50075, ipcPort=50020):DataXceiver
java.net.SocketTimeoutException: 480000 millis timeout while waiting
for channel to be ready for write. ch :
java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
remote=/192.168.1.148:37188]
	at org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
	at org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
	at org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
	at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
	at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
	at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
	at java.lang.Thread.run(Thread.java:662)
2013-02-22 11:12:41,892 INFO
org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
succeeded for blk_-2674357249542194287_43419


Here is our env:
hadoop 1.0.3
hbase 0.94.1(snappy enabled)

java version "1.6.0_31"
Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)

# ulimit -a
core file size          (blocks, -c) 0
data seg size           (kbytes, -d) unlimited
scheduling priority             (-e) 20
file size               (blocks, -f) unlimited
pending signals                 (-i) 16382
max locked memory       (kbytes, -l) 64
max memory size         (kbytes, -m) unlimited
open files                      (-n) 32768
pipe size            (512 bytes, -p) 8
POSIX message queues     (bytes, -q) 819200
real-time priority              (-r) 0
stack size              (kbytes, -s) 8192
cpu time               (seconds, -t) unlimited
max user processes              (-u) 32768
virtual memory          (kbytes, -v) unlimited
file locks                      (-x) unlimited

# uname -a
Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
UTC 2011 x86_64 GNU/Linux


# free(master)
             total       used       free     shared    buffers     cached
Mem:      24732936    8383708   16349228          0     490584    2580356
-/+ buffers/cache:    5312768   19420168
Swap:     72458232          0   72458232


# free(slaves)
             total       used       free     shared    buffers     cached
Mem:      24733000   22824276    1908724          0     862556   15303304
-/+ buffers/cache:    6658416   18074584
Swap:     72458232        264   72457968


Some important conf:
core-site.xml
	<property>
		<name>io.file.buffer.size</name>
		<value>65536</value>
	</property>

hdfs-site.xml
	<property>
		<name>dfs.block.size</name>
		<value>134217728</value>
	</property>
	<property>
		<name>dfs.datanode.max.xcievers</name>
		<value>4096</value>
	</property>
	<property>
		<name>dfs.support.append</name>
		<value>true</value>
	</property>
	<property>
		<name>dfs.replication</name>
		<value>2</value>
	</property>


Hope you can help us.
Thanks in advance.



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Yes, we make sure that inappropriate use of NFS leading to high load
and the lost heartbeat between cluster members.
There was a NFS partition point to one virtual machine for some
purpose, but the virtual machine shutted down frequently.
BTW, the NFS partition was not for the backup of NN metadata, just for
other temporary purpose, and it has been removed now.
The NFS partition(with autofs) for NN metadata backup has no problem.

For more info, google the "NFS high load"...


On Wed, Feb 27, 2013 at 9:58 AM, Jean-Marc Spaggiari
<je...@spaggiari.org> wrote:
> Hi Davey,
>
> So were you able to find the issue?
>
> JM
>
> 2013/2/25 Davey Yan <da...@gmail.com>:
>> Hi Nicolas,
>>
>> I think i found what led to shutdown of all of the datanodes, but i am
>> not completely certain.
>> I will return to this mail list when my cluster returns to be stable.
>>
>> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>>> Network error messages are not always friendly, especially if there is a
>>> misconfiguration.
>>> This said,  "connection refused" says that the network connection was made,
>>> but that the remote port was not opened on the remote box. I.e. the process
>>> was dead.
>>> It could be useful to pastebin the whole logs as well...
>>>
>>>
>>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>>
>>>> But... there was no log like "network unreachable".
>>>>
>>>>
>>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>>> wrote:
>>>> > I agree.
>>>> > Then for HDFS, ...
>>>> > The first thing to check is the network I would say.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>>> >>
>>>> >> Thanks for reply, Nicolas.
>>>> >>
>>>> >> My question: What can lead to shutdown of all of the datanodes?
>>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>>> >>
>>>> >>
>>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>>> >> wrote:
>>>> >> > Ok, what's your question?
>>>> >> > When you say the datanode went down, was it the datanode processes or
>>>> >> > the
>>>> >> > machines, with both the datanodes and the regionservers?
>>>> >> >
>>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>>> >> > internally
>>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>>> >> > you
>>>> >> > have
>>>> >> > 'no answer for x minutes').
>>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>>> >> > considered
>>>> >> > as live.
>>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>>> >> > (i.e.
>>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>>> >> > a
>>>> >> > high
>>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>>> >> > refused
>>>> >> > &
>>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>>> >> > does
>>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>>> >> > obviously,
>>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>>> >> > datanodes,
>>>> >> > called "stale": an intermediary state where the datanode is used only
>>>> >> > if
>>>> >> > you
>>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>>> >> > It
>>>> >> > will
>>>> >> > be documented in HBase for the 0.96 release. But if all your
>>>> >> > datanodes
>>>> >> > are
>>>> >> > down it won't change much.
>>>> >> >
>>>> >> > Cheers,
>>>> >> >
>>>> >> > Nicolas
>>>> >> >
>>>> >> >
>>>> >> >
>>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>>> >> > wrote:
>>>> >> >>
>>>> >> >> Hey guys,
>>>> >> >>
>>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>>> >> >> 1
>>>> >> >> year, and it works fine.
>>>> >> >> But the datanodes got shutdown twice in the last month.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>>> >> >> but regionservers of HBase were still live in the HBase web
>>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>>> >> >> All of the processes of jvm were still running, including
>>>> >> >> hmaster/namenode/regionserver/datanode.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>>> >> >> of
>>>> >> >> slaves became very high, more than 10, higher than normal running.
>>>> >> >> From the "top" command, we saw that the processes of datanode and
>>>> >> >> regionserver were comsuming CPU.
>>>> >> >>
>>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>>> >> >> load of slaves returned to normal level, and we start the cluster
>>>> >> >> again.
>>>> >> >>
>>>> >> >>
>>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.148:50010
>>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>>> >> >>
>>>> >> >>
>>>> >> >> Logs in DNs indicated there were many IOException and
>>>> >> >> SocketTimeoutException:
>>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.io.IOException: Interrupted receiveBlock
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:03:44,823 WARN
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:48654]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:37188]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:12:41,892 INFO
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>>> >> >> Verification
>>>> >> >> succeeded for blk_-2674357249542194287_43419
>>>> >> >>
>>>> >> >>
>>>> >> >> Here is our env:
>>>> >> >> hadoop 1.0.3
>>>> >> >> hbase 0.94.1(snappy enabled)
>>>> >> >>
>>>> >> >> java version "1.6.0_31"
>>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>>> >> >>
>>>> >> >> # ulimit -a
>>>> >> >> core file size          (blocks, -c) 0
>>>> >> >> data seg size           (kbytes, -d) unlimited
>>>> >> >> scheduling priority             (-e) 20
>>>> >> >> file size               (blocks, -f) unlimited
>>>> >> >> pending signals                 (-i) 16382
>>>> >> >> max locked memory       (kbytes, -l) 64
>>>> >> >> max memory size         (kbytes, -m) unlimited
>>>> >> >> open files                      (-n) 32768
>>>> >> >> pipe size            (512 bytes, -p) 8
>>>> >> >> POSIX message queues     (bytes, -q) 819200
>>>> >> >> real-time priority              (-r) 0
>>>> >> >> stack size              (kbytes, -s) 8192
>>>> >> >> cpu time               (seconds, -t) unlimited
>>>> >> >> max user processes              (-u) 32768
>>>> >> >> virtual memory          (kbytes, -v) unlimited
>>>> >> >> file locks                      (-x) unlimited
>>>> >> >>
>>>> >> >> # uname -a
>>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>>> >> >> UTC 2011 x86_64 GNU/Linux
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(master)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>>> >> >> 2580356
>>>> >> >> -/+ buffers/cache:    5312768   19420168
>>>> >> >> Swap:     72458232          0   72458232
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(slaves)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>>> >> >> 15303304
>>>> >> >> -/+ buffers/cache:    6658416   18074584
>>>> >> >> Swap:     72458232        264   72457968
>>>> >> >>
>>>> >> >>
>>>> >> >> Some important conf:
>>>> >> >> core-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>io.file.buffer.size</name>
>>>> >> >>                 <value>65536</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >> hdfs-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.block.size</name>
>>>> >> >>                 <value>134217728</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>>> >> >>                 <value>4096</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.support.append</name>
>>>> >> >>                 <value>true</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.replication</name>
>>>> >> >>                 <value>2</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >>
>>>> >> >> Hope you can help us.
>>>> >> >> Thanks in advance.
>>>> >> >>
>>>> >> >>
>>>> >> >>
>>>> >> >> --
>>>> >> >> Davey Yan
>>>> >> >
>>>> >> >
>>>> >>
>>>> >>
>>>> >>
>>>> >> --
>>>> >> Davey Yan
>>>> >
>>>> >
>>>>
>>>>
>>>>
>>>> --
>>>> Davey Yan
>>>
>>>
>>
>>
>>
>> --
>> Davey Yan



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Yes, we make sure that inappropriate use of NFS leading to high load
and the lost heartbeat between cluster members.
There was a NFS partition point to one virtual machine for some
purpose, but the virtual machine shutted down frequently.
BTW, the NFS partition was not for the backup of NN metadata, just for
other temporary purpose, and it has been removed now.
The NFS partition(with autofs) for NN metadata backup has no problem.

For more info, google the "NFS high load"...


On Wed, Feb 27, 2013 at 9:58 AM, Jean-Marc Spaggiari
<je...@spaggiari.org> wrote:
> Hi Davey,
>
> So were you able to find the issue?
>
> JM
>
> 2013/2/25 Davey Yan <da...@gmail.com>:
>> Hi Nicolas,
>>
>> I think i found what led to shutdown of all of the datanodes, but i am
>> not completely certain.
>> I will return to this mail list when my cluster returns to be stable.
>>
>> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>>> Network error messages are not always friendly, especially if there is a
>>> misconfiguration.
>>> This said,  "connection refused" says that the network connection was made,
>>> but that the remote port was not opened on the remote box. I.e. the process
>>> was dead.
>>> It could be useful to pastebin the whole logs as well...
>>>
>>>
>>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>>
>>>> But... there was no log like "network unreachable".
>>>>
>>>>
>>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>>> wrote:
>>>> > I agree.
>>>> > Then for HDFS, ...
>>>> > The first thing to check is the network I would say.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>>> >>
>>>> >> Thanks for reply, Nicolas.
>>>> >>
>>>> >> My question: What can lead to shutdown of all of the datanodes?
>>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>>> >>
>>>> >>
>>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>>> >> wrote:
>>>> >> > Ok, what's your question?
>>>> >> > When you say the datanode went down, was it the datanode processes or
>>>> >> > the
>>>> >> > machines, with both the datanodes and the regionservers?
>>>> >> >
>>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>>> >> > internally
>>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>>> >> > you
>>>> >> > have
>>>> >> > 'no answer for x minutes').
>>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>>> >> > considered
>>>> >> > as live.
>>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>>> >> > (i.e.
>>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>>> >> > a
>>>> >> > high
>>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>>> >> > refused
>>>> >> > &
>>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>>> >> > does
>>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>>> >> > obviously,
>>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>>> >> > datanodes,
>>>> >> > called "stale": an intermediary state where the datanode is used only
>>>> >> > if
>>>> >> > you
>>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>>> >> > It
>>>> >> > will
>>>> >> > be documented in HBase for the 0.96 release. But if all your
>>>> >> > datanodes
>>>> >> > are
>>>> >> > down it won't change much.
>>>> >> >
>>>> >> > Cheers,
>>>> >> >
>>>> >> > Nicolas
>>>> >> >
>>>> >> >
>>>> >> >
>>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>>> >> > wrote:
>>>> >> >>
>>>> >> >> Hey guys,
>>>> >> >>
>>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>>> >> >> 1
>>>> >> >> year, and it works fine.
>>>> >> >> But the datanodes got shutdown twice in the last month.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>>> >> >> but regionservers of HBase were still live in the HBase web
>>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>>> >> >> All of the processes of jvm were still running, including
>>>> >> >> hmaster/namenode/regionserver/datanode.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>>> >> >> of
>>>> >> >> slaves became very high, more than 10, higher than normal running.
>>>> >> >> From the "top" command, we saw that the processes of datanode and
>>>> >> >> regionserver were comsuming CPU.
>>>> >> >>
>>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>>> >> >> load of slaves returned to normal level, and we start the cluster
>>>> >> >> again.
>>>> >> >>
>>>> >> >>
>>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.148:50010
>>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>>> >> >>
>>>> >> >>
>>>> >> >> Logs in DNs indicated there were many IOException and
>>>> >> >> SocketTimeoutException:
>>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.io.IOException: Interrupted receiveBlock
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:03:44,823 WARN
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:48654]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:37188]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:12:41,892 INFO
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>>> >> >> Verification
>>>> >> >> succeeded for blk_-2674357249542194287_43419
>>>> >> >>
>>>> >> >>
>>>> >> >> Here is our env:
>>>> >> >> hadoop 1.0.3
>>>> >> >> hbase 0.94.1(snappy enabled)
>>>> >> >>
>>>> >> >> java version "1.6.0_31"
>>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>>> >> >>
>>>> >> >> # ulimit -a
>>>> >> >> core file size          (blocks, -c) 0
>>>> >> >> data seg size           (kbytes, -d) unlimited
>>>> >> >> scheduling priority             (-e) 20
>>>> >> >> file size               (blocks, -f) unlimited
>>>> >> >> pending signals                 (-i) 16382
>>>> >> >> max locked memory       (kbytes, -l) 64
>>>> >> >> max memory size         (kbytes, -m) unlimited
>>>> >> >> open files                      (-n) 32768
>>>> >> >> pipe size            (512 bytes, -p) 8
>>>> >> >> POSIX message queues     (bytes, -q) 819200
>>>> >> >> real-time priority              (-r) 0
>>>> >> >> stack size              (kbytes, -s) 8192
>>>> >> >> cpu time               (seconds, -t) unlimited
>>>> >> >> max user processes              (-u) 32768
>>>> >> >> virtual memory          (kbytes, -v) unlimited
>>>> >> >> file locks                      (-x) unlimited
>>>> >> >>
>>>> >> >> # uname -a
>>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>>> >> >> UTC 2011 x86_64 GNU/Linux
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(master)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>>> >> >> 2580356
>>>> >> >> -/+ buffers/cache:    5312768   19420168
>>>> >> >> Swap:     72458232          0   72458232
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(slaves)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>>> >> >> 15303304
>>>> >> >> -/+ buffers/cache:    6658416   18074584
>>>> >> >> Swap:     72458232        264   72457968
>>>> >> >>
>>>> >> >>
>>>> >> >> Some important conf:
>>>> >> >> core-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>io.file.buffer.size</name>
>>>> >> >>                 <value>65536</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >> hdfs-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.block.size</name>
>>>> >> >>                 <value>134217728</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>>> >> >>                 <value>4096</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.support.append</name>
>>>> >> >>                 <value>true</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.replication</name>
>>>> >> >>                 <value>2</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >>
>>>> >> >> Hope you can help us.
>>>> >> >> Thanks in advance.
>>>> >> >>
>>>> >> >>
>>>> >> >>
>>>> >> >> --
>>>> >> >> Davey Yan
>>>> >> >
>>>> >> >
>>>> >>
>>>> >>
>>>> >>
>>>> >> --
>>>> >> Davey Yan
>>>> >
>>>> >
>>>>
>>>>
>>>>
>>>> --
>>>> Davey Yan
>>>
>>>
>>
>>
>>
>> --
>> Davey Yan



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Yes, we make sure that inappropriate use of NFS leading to high load
and the lost heartbeat between cluster members.
There was a NFS partition point to one virtual machine for some
purpose, but the virtual machine shutted down frequently.
BTW, the NFS partition was not for the backup of NN metadata, just for
other temporary purpose, and it has been removed now.
The NFS partition(with autofs) for NN metadata backup has no problem.

For more info, google the "NFS high load"...


On Wed, Feb 27, 2013 at 9:58 AM, Jean-Marc Spaggiari
<je...@spaggiari.org> wrote:
> Hi Davey,
>
> So were you able to find the issue?
>
> JM
>
> 2013/2/25 Davey Yan <da...@gmail.com>:
>> Hi Nicolas,
>>
>> I think i found what led to shutdown of all of the datanodes, but i am
>> not completely certain.
>> I will return to this mail list when my cluster returns to be stable.
>>
>> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>>> Network error messages are not always friendly, especially if there is a
>>> misconfiguration.
>>> This said,  "connection refused" says that the network connection was made,
>>> but that the remote port was not opened on the remote box. I.e. the process
>>> was dead.
>>> It could be useful to pastebin the whole logs as well...
>>>
>>>
>>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>>
>>>> But... there was no log like "network unreachable".
>>>>
>>>>
>>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>>> wrote:
>>>> > I agree.
>>>> > Then for HDFS, ...
>>>> > The first thing to check is the network I would say.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>>> >>
>>>> >> Thanks for reply, Nicolas.
>>>> >>
>>>> >> My question: What can lead to shutdown of all of the datanodes?
>>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>>> >>
>>>> >>
>>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>>> >> wrote:
>>>> >> > Ok, what's your question?
>>>> >> > When you say the datanode went down, was it the datanode processes or
>>>> >> > the
>>>> >> > machines, with both the datanodes and the regionservers?
>>>> >> >
>>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>>> >> > internally
>>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>>> >> > you
>>>> >> > have
>>>> >> > 'no answer for x minutes').
>>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>>> >> > considered
>>>> >> > as live.
>>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>>> >> > (i.e.
>>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>>> >> > a
>>>> >> > high
>>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>>> >> > refused
>>>> >> > &
>>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>>> >> > does
>>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>>> >> > obviously,
>>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>>> >> > datanodes,
>>>> >> > called "stale": an intermediary state where the datanode is used only
>>>> >> > if
>>>> >> > you
>>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>>> >> > It
>>>> >> > will
>>>> >> > be documented in HBase for the 0.96 release. But if all your
>>>> >> > datanodes
>>>> >> > are
>>>> >> > down it won't change much.
>>>> >> >
>>>> >> > Cheers,
>>>> >> >
>>>> >> > Nicolas
>>>> >> >
>>>> >> >
>>>> >> >
>>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>>> >> > wrote:
>>>> >> >>
>>>> >> >> Hey guys,
>>>> >> >>
>>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>>> >> >> 1
>>>> >> >> year, and it works fine.
>>>> >> >> But the datanodes got shutdown twice in the last month.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>>> >> >> but regionservers of HBase were still live in the HBase web
>>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>>> >> >> All of the processes of jvm were still running, including
>>>> >> >> hmaster/namenode/regionserver/datanode.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>>> >> >> of
>>>> >> >> slaves became very high, more than 10, higher than normal running.
>>>> >> >> From the "top" command, we saw that the processes of datanode and
>>>> >> >> regionserver were comsuming CPU.
>>>> >> >>
>>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>>> >> >> load of slaves returned to normal level, and we start the cluster
>>>> >> >> again.
>>>> >> >>
>>>> >> >>
>>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.148:50010
>>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>>> >> >>
>>>> >> >>
>>>> >> >> Logs in DNs indicated there were many IOException and
>>>> >> >> SocketTimeoutException:
>>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.io.IOException: Interrupted receiveBlock
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:03:44,823 WARN
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:48654]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:37188]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:12:41,892 INFO
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>>> >> >> Verification
>>>> >> >> succeeded for blk_-2674357249542194287_43419
>>>> >> >>
>>>> >> >>
>>>> >> >> Here is our env:
>>>> >> >> hadoop 1.0.3
>>>> >> >> hbase 0.94.1(snappy enabled)
>>>> >> >>
>>>> >> >> java version "1.6.0_31"
>>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>>> >> >>
>>>> >> >> # ulimit -a
>>>> >> >> core file size          (blocks, -c) 0
>>>> >> >> data seg size           (kbytes, -d) unlimited
>>>> >> >> scheduling priority             (-e) 20
>>>> >> >> file size               (blocks, -f) unlimited
>>>> >> >> pending signals                 (-i) 16382
>>>> >> >> max locked memory       (kbytes, -l) 64
>>>> >> >> max memory size         (kbytes, -m) unlimited
>>>> >> >> open files                      (-n) 32768
>>>> >> >> pipe size            (512 bytes, -p) 8
>>>> >> >> POSIX message queues     (bytes, -q) 819200
>>>> >> >> real-time priority              (-r) 0
>>>> >> >> stack size              (kbytes, -s) 8192
>>>> >> >> cpu time               (seconds, -t) unlimited
>>>> >> >> max user processes              (-u) 32768
>>>> >> >> virtual memory          (kbytes, -v) unlimited
>>>> >> >> file locks                      (-x) unlimited
>>>> >> >>
>>>> >> >> # uname -a
>>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>>> >> >> UTC 2011 x86_64 GNU/Linux
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(master)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>>> >> >> 2580356
>>>> >> >> -/+ buffers/cache:    5312768   19420168
>>>> >> >> Swap:     72458232          0   72458232
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(slaves)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>>> >> >> 15303304
>>>> >> >> -/+ buffers/cache:    6658416   18074584
>>>> >> >> Swap:     72458232        264   72457968
>>>> >> >>
>>>> >> >>
>>>> >> >> Some important conf:
>>>> >> >> core-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>io.file.buffer.size</name>
>>>> >> >>                 <value>65536</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >> hdfs-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.block.size</name>
>>>> >> >>                 <value>134217728</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>>> >> >>                 <value>4096</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.support.append</name>
>>>> >> >>                 <value>true</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.replication</name>
>>>> >> >>                 <value>2</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >>
>>>> >> >> Hope you can help us.
>>>> >> >> Thanks in advance.
>>>> >> >>
>>>> >> >>
>>>> >> >>
>>>> >> >> --
>>>> >> >> Davey Yan
>>>> >> >
>>>> >> >
>>>> >>
>>>> >>
>>>> >>
>>>> >> --
>>>> >> Davey Yan
>>>> >
>>>> >
>>>>
>>>>
>>>>
>>>> --
>>>> Davey Yan
>>>
>>>
>>
>>
>>
>> --
>> Davey Yan



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Yes, we make sure that inappropriate use of NFS leading to high load
and the lost heartbeat between cluster members.
There was a NFS partition point to one virtual machine for some
purpose, but the virtual machine shutted down frequently.
BTW, the NFS partition was not for the backup of NN metadata, just for
other temporary purpose, and it has been removed now.
The NFS partition(with autofs) for NN metadata backup has no problem.

For more info, google the "NFS high load"...


On Wed, Feb 27, 2013 at 9:58 AM, Jean-Marc Spaggiari
<je...@spaggiari.org> wrote:
> Hi Davey,
>
> So were you able to find the issue?
>
> JM
>
> 2013/2/25 Davey Yan <da...@gmail.com>:
>> Hi Nicolas,
>>
>> I think i found what led to shutdown of all of the datanodes, but i am
>> not completely certain.
>> I will return to this mail list when my cluster returns to be stable.
>>
>> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>>> Network error messages are not always friendly, especially if there is a
>>> misconfiguration.
>>> This said,  "connection refused" says that the network connection was made,
>>> but that the remote port was not opened on the remote box. I.e. the process
>>> was dead.
>>> It could be useful to pastebin the whole logs as well...
>>>
>>>
>>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>>
>>>> But... there was no log like "network unreachable".
>>>>
>>>>
>>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>>> wrote:
>>>> > I agree.
>>>> > Then for HDFS, ...
>>>> > The first thing to check is the network I would say.
>>>> >
>>>> >
>>>> >
>>>> >
>>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>>> >>
>>>> >> Thanks for reply, Nicolas.
>>>> >>
>>>> >> My question: What can lead to shutdown of all of the datanodes?
>>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>>> >>
>>>> >>
>>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>>> >> wrote:
>>>> >> > Ok, what's your question?
>>>> >> > When you say the datanode went down, was it the datanode processes or
>>>> >> > the
>>>> >> > machines, with both the datanodes and the regionservers?
>>>> >> >
>>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>>> >> > internally
>>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>>> >> > you
>>>> >> > have
>>>> >> > 'no answer for x minutes').
>>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>>> >> > considered
>>>> >> > as live.
>>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>>> >> > (i.e.
>>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>>> >> > a
>>>> >> > high
>>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>>> >> > refused
>>>> >> > &
>>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>>> >> > does
>>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>>> >> > obviously,
>>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>>> >> > datanodes,
>>>> >> > called "stale": an intermediary state where the datanode is used only
>>>> >> > if
>>>> >> > you
>>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>>> >> > It
>>>> >> > will
>>>> >> > be documented in HBase for the 0.96 release. But if all your
>>>> >> > datanodes
>>>> >> > are
>>>> >> > down it won't change much.
>>>> >> >
>>>> >> > Cheers,
>>>> >> >
>>>> >> > Nicolas
>>>> >> >
>>>> >> >
>>>> >> >
>>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>>> >> > wrote:
>>>> >> >>
>>>> >> >> Hey guys,
>>>> >> >>
>>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>>> >> >> 1
>>>> >> >> year, and it works fine.
>>>> >> >> But the datanodes got shutdown twice in the last month.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>>> >> >> but regionservers of HBase were still live in the HBase web
>>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>>> >> >> All of the processes of jvm were still running, including
>>>> >> >> hmaster/namenode/regionserver/datanode.
>>>> >> >>
>>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>>> >> >> of
>>>> >> >> slaves became very high, more than 10, higher than normal running.
>>>> >> >> From the "top" command, we saw that the processes of datanode and
>>>> >> >> regionserver were comsuming CPU.
>>>> >> >>
>>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>>> >> >> load of slaves returned to normal level, and we start the cluster
>>>> >> >> again.
>>>> >> >>
>>>> >> >>
>>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>>> >> >> 192.168.1.148:50010
>>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>>> >> >>
>>>> >> >>
>>>> >> >> Logs in DNs indicated there were many IOException and
>>>> >> >> SocketTimeoutException:
>>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.io.IOException: Interrupted receiveBlock
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:03:44,823 WARN
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:48654]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>>> >> >> for channel to be ready for write. ch :
>>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>>> >> >> remote=/192.168.1.148:37188]
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>>> >> >>         at
>>>> >> >>
>>>> >> >>
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>>> >> >> 2013-02-22 11:12:41,892 INFO
>>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>>> >> >> Verification
>>>> >> >> succeeded for blk_-2674357249542194287_43419
>>>> >> >>
>>>> >> >>
>>>> >> >> Here is our env:
>>>> >> >> hadoop 1.0.3
>>>> >> >> hbase 0.94.1(snappy enabled)
>>>> >> >>
>>>> >> >> java version "1.6.0_31"
>>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>>> >> >>
>>>> >> >> # ulimit -a
>>>> >> >> core file size          (blocks, -c) 0
>>>> >> >> data seg size           (kbytes, -d) unlimited
>>>> >> >> scheduling priority             (-e) 20
>>>> >> >> file size               (blocks, -f) unlimited
>>>> >> >> pending signals                 (-i) 16382
>>>> >> >> max locked memory       (kbytes, -l) 64
>>>> >> >> max memory size         (kbytes, -m) unlimited
>>>> >> >> open files                      (-n) 32768
>>>> >> >> pipe size            (512 bytes, -p) 8
>>>> >> >> POSIX message queues     (bytes, -q) 819200
>>>> >> >> real-time priority              (-r) 0
>>>> >> >> stack size              (kbytes, -s) 8192
>>>> >> >> cpu time               (seconds, -t) unlimited
>>>> >> >> max user processes              (-u) 32768
>>>> >> >> virtual memory          (kbytes, -v) unlimited
>>>> >> >> file locks                      (-x) unlimited
>>>> >> >>
>>>> >> >> # uname -a
>>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>>> >> >> UTC 2011 x86_64 GNU/Linux
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(master)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>>> >> >> 2580356
>>>> >> >> -/+ buffers/cache:    5312768   19420168
>>>> >> >> Swap:     72458232          0   72458232
>>>> >> >>
>>>> >> >>
>>>> >> >> # free(slaves)
>>>> >> >>              total       used       free     shared    buffers
>>>> >> >> cached
>>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>>> >> >> 15303304
>>>> >> >> -/+ buffers/cache:    6658416   18074584
>>>> >> >> Swap:     72458232        264   72457968
>>>> >> >>
>>>> >> >>
>>>> >> >> Some important conf:
>>>> >> >> core-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>io.file.buffer.size</name>
>>>> >> >>                 <value>65536</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >> hdfs-site.xml
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.block.size</name>
>>>> >> >>                 <value>134217728</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>>> >> >>                 <value>4096</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.support.append</name>
>>>> >> >>                 <value>true</value>
>>>> >> >>         </property>
>>>> >> >>         <property>
>>>> >> >>                 <name>dfs.replication</name>
>>>> >> >>                 <value>2</value>
>>>> >> >>         </property>
>>>> >> >>
>>>> >> >>
>>>> >> >> Hope you can help us.
>>>> >> >> Thanks in advance.
>>>> >> >>
>>>> >> >>
>>>> >> >>
>>>> >> >> --
>>>> >> >> Davey Yan
>>>> >> >
>>>> >> >
>>>> >>
>>>> >>
>>>> >>
>>>> >> --
>>>> >> Davey Yan
>>>> >
>>>> >
>>>>
>>>>
>>>>
>>>> --
>>>> Davey Yan
>>>
>>>
>>
>>
>>
>> --
>> Davey Yan



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Hi Davey,

So were you able to find the issue?

JM

2013/2/25 Davey Yan <da...@gmail.com>:
> Hi Nicolas,
>
> I think i found what led to shutdown of all of the datanodes, but i am
> not completely certain.
> I will return to this mail list when my cluster returns to be stable.
>
> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>> Network error messages are not always friendly, especially if there is a
>> misconfiguration.
>> This said,  "connection refused" says that the network connection was made,
>> but that the remote port was not opened on the remote box. I.e. the process
>> was dead.
>> It could be useful to pastebin the whole logs as well...
>>
>>
>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>
>>> But... there was no log like "network unreachable".
>>>
>>>
>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>> wrote:
>>> > I agree.
>>> > Then for HDFS, ...
>>> > The first thing to check is the network I would say.
>>> >
>>> >
>>> >
>>> >
>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>> >>
>>> >> Thanks for reply, Nicolas.
>>> >>
>>> >> My question: What can lead to shutdown of all of the datanodes?
>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>> >>
>>> >>
>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>> >> wrote:
>>> >> > Ok, what's your question?
>>> >> > When you say the datanode went down, was it the datanode processes or
>>> >> > the
>>> >> > machines, with both the datanodes and the regionservers?
>>> >> >
>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>> >> > internally
>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>> >> > you
>>> >> > have
>>> >> > 'no answer for x minutes').
>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>> >> > considered
>>> >> > as live.
>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>> >> > (i.e.
>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>> >> > a
>>> >> > high
>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>> >> > refused
>>> >> > &
>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>> >> > does
>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>> >> > obviously,
>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>> >> > datanodes,
>>> >> > called "stale": an intermediary state where the datanode is used only
>>> >> > if
>>> >> > you
>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>> >> > It
>>> >> > will
>>> >> > be documented in HBase for the 0.96 release. But if all your
>>> >> > datanodes
>>> >> > are
>>> >> > down it won't change much.
>>> >> >
>>> >> > Cheers,
>>> >> >
>>> >> > Nicolas
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>> >> > wrote:
>>> >> >>
>>> >> >> Hey guys,
>>> >> >>
>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>> >> >> 1
>>> >> >> year, and it works fine.
>>> >> >> But the datanodes got shutdown twice in the last month.
>>> >> >>
>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>> >> >> but regionservers of HBase were still live in the HBase web
>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>> >> >> All of the processes of jvm were still running, including
>>> >> >> hmaster/namenode/regionserver/datanode.
>>> >> >>
>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>> >> >> of
>>> >> >> slaves became very high, more than 10, higher than normal running.
>>> >> >> From the "top" command, we saw that the processes of datanode and
>>> >> >> regionserver were comsuming CPU.
>>> >> >>
>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>> >> >> load of slaves returned to normal level, and we start the cluster
>>> >> >> again.
>>> >> >>
>>> >> >>
>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.148:50010
>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>> >> >>
>>> >> >>
>>> >> >> Logs in DNs indicated there were many IOException and
>>> >> >> SocketTimeoutException:
>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.io.IOException: Interrupted receiveBlock
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:03:44,823 WARN
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:48654]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:37188]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:12:41,892 INFO
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>> >> >> Verification
>>> >> >> succeeded for blk_-2674357249542194287_43419
>>> >> >>
>>> >> >>
>>> >> >> Here is our env:
>>> >> >> hadoop 1.0.3
>>> >> >> hbase 0.94.1(snappy enabled)
>>> >> >>
>>> >> >> java version "1.6.0_31"
>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>> >> >>
>>> >> >> # ulimit -a
>>> >> >> core file size          (blocks, -c) 0
>>> >> >> data seg size           (kbytes, -d) unlimited
>>> >> >> scheduling priority             (-e) 20
>>> >> >> file size               (blocks, -f) unlimited
>>> >> >> pending signals                 (-i) 16382
>>> >> >> max locked memory       (kbytes, -l) 64
>>> >> >> max memory size         (kbytes, -m) unlimited
>>> >> >> open files                      (-n) 32768
>>> >> >> pipe size            (512 bytes, -p) 8
>>> >> >> POSIX message queues     (bytes, -q) 819200
>>> >> >> real-time priority              (-r) 0
>>> >> >> stack size              (kbytes, -s) 8192
>>> >> >> cpu time               (seconds, -t) unlimited
>>> >> >> max user processes              (-u) 32768
>>> >> >> virtual memory          (kbytes, -v) unlimited
>>> >> >> file locks                      (-x) unlimited
>>> >> >>
>>> >> >> # uname -a
>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>> >> >> UTC 2011 x86_64 GNU/Linux
>>> >> >>
>>> >> >>
>>> >> >> # free(master)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>> >> >> 2580356
>>> >> >> -/+ buffers/cache:    5312768   19420168
>>> >> >> Swap:     72458232          0   72458232
>>> >> >>
>>> >> >>
>>> >> >> # free(slaves)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>> >> >> 15303304
>>> >> >> -/+ buffers/cache:    6658416   18074584
>>> >> >> Swap:     72458232        264   72457968
>>> >> >>
>>> >> >>
>>> >> >> Some important conf:
>>> >> >> core-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>io.file.buffer.size</name>
>>> >> >>                 <value>65536</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >> hdfs-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>dfs.block.size</name>
>>> >> >>                 <value>134217728</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>> >> >>                 <value>4096</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.support.append</name>
>>> >> >>                 <value>true</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.replication</name>
>>> >> >>                 <value>2</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >>
>>> >> >> Hope you can help us.
>>> >> >> Thanks in advance.
>>> >> >>
>>> >> >>
>>> >> >>
>>> >> >> --
>>> >> >> Davey Yan
>>> >> >
>>> >> >
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Davey Yan
>>> >
>>> >
>>>
>>>
>>>
>>> --
>>> Davey Yan
>>
>>
>
>
>
> --
> Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Hi Davey,

So were you able to find the issue?

JM

2013/2/25 Davey Yan <da...@gmail.com>:
> Hi Nicolas,
>
> I think i found what led to shutdown of all of the datanodes, but i am
> not completely certain.
> I will return to this mail list when my cluster returns to be stable.
>
> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>> Network error messages are not always friendly, especially if there is a
>> misconfiguration.
>> This said,  "connection refused" says that the network connection was made,
>> but that the remote port was not opened on the remote box. I.e. the process
>> was dead.
>> It could be useful to pastebin the whole logs as well...
>>
>>
>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>
>>> But... there was no log like "network unreachable".
>>>
>>>
>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>> wrote:
>>> > I agree.
>>> > Then for HDFS, ...
>>> > The first thing to check is the network I would say.
>>> >
>>> >
>>> >
>>> >
>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>> >>
>>> >> Thanks for reply, Nicolas.
>>> >>
>>> >> My question: What can lead to shutdown of all of the datanodes?
>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>> >>
>>> >>
>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>> >> wrote:
>>> >> > Ok, what's your question?
>>> >> > When you say the datanode went down, was it the datanode processes or
>>> >> > the
>>> >> > machines, with both the datanodes and the regionservers?
>>> >> >
>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>> >> > internally
>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>> >> > you
>>> >> > have
>>> >> > 'no answer for x minutes').
>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>> >> > considered
>>> >> > as live.
>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>> >> > (i.e.
>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>> >> > a
>>> >> > high
>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>> >> > refused
>>> >> > &
>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>> >> > does
>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>> >> > obviously,
>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>> >> > datanodes,
>>> >> > called "stale": an intermediary state where the datanode is used only
>>> >> > if
>>> >> > you
>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>> >> > It
>>> >> > will
>>> >> > be documented in HBase for the 0.96 release. But if all your
>>> >> > datanodes
>>> >> > are
>>> >> > down it won't change much.
>>> >> >
>>> >> > Cheers,
>>> >> >
>>> >> > Nicolas
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>> >> > wrote:
>>> >> >>
>>> >> >> Hey guys,
>>> >> >>
>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>> >> >> 1
>>> >> >> year, and it works fine.
>>> >> >> But the datanodes got shutdown twice in the last month.
>>> >> >>
>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>> >> >> but regionservers of HBase were still live in the HBase web
>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>> >> >> All of the processes of jvm were still running, including
>>> >> >> hmaster/namenode/regionserver/datanode.
>>> >> >>
>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>> >> >> of
>>> >> >> slaves became very high, more than 10, higher than normal running.
>>> >> >> From the "top" command, we saw that the processes of datanode and
>>> >> >> regionserver were comsuming CPU.
>>> >> >>
>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>> >> >> load of slaves returned to normal level, and we start the cluster
>>> >> >> again.
>>> >> >>
>>> >> >>
>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.148:50010
>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>> >> >>
>>> >> >>
>>> >> >> Logs in DNs indicated there were many IOException and
>>> >> >> SocketTimeoutException:
>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.io.IOException: Interrupted receiveBlock
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:03:44,823 WARN
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:48654]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:37188]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:12:41,892 INFO
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>> >> >> Verification
>>> >> >> succeeded for blk_-2674357249542194287_43419
>>> >> >>
>>> >> >>
>>> >> >> Here is our env:
>>> >> >> hadoop 1.0.3
>>> >> >> hbase 0.94.1(snappy enabled)
>>> >> >>
>>> >> >> java version "1.6.0_31"
>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>> >> >>
>>> >> >> # ulimit -a
>>> >> >> core file size          (blocks, -c) 0
>>> >> >> data seg size           (kbytes, -d) unlimited
>>> >> >> scheduling priority             (-e) 20
>>> >> >> file size               (blocks, -f) unlimited
>>> >> >> pending signals                 (-i) 16382
>>> >> >> max locked memory       (kbytes, -l) 64
>>> >> >> max memory size         (kbytes, -m) unlimited
>>> >> >> open files                      (-n) 32768
>>> >> >> pipe size            (512 bytes, -p) 8
>>> >> >> POSIX message queues     (bytes, -q) 819200
>>> >> >> real-time priority              (-r) 0
>>> >> >> stack size              (kbytes, -s) 8192
>>> >> >> cpu time               (seconds, -t) unlimited
>>> >> >> max user processes              (-u) 32768
>>> >> >> virtual memory          (kbytes, -v) unlimited
>>> >> >> file locks                      (-x) unlimited
>>> >> >>
>>> >> >> # uname -a
>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>> >> >> UTC 2011 x86_64 GNU/Linux
>>> >> >>
>>> >> >>
>>> >> >> # free(master)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>> >> >> 2580356
>>> >> >> -/+ buffers/cache:    5312768   19420168
>>> >> >> Swap:     72458232          0   72458232
>>> >> >>
>>> >> >>
>>> >> >> # free(slaves)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>> >> >> 15303304
>>> >> >> -/+ buffers/cache:    6658416   18074584
>>> >> >> Swap:     72458232        264   72457968
>>> >> >>
>>> >> >>
>>> >> >> Some important conf:
>>> >> >> core-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>io.file.buffer.size</name>
>>> >> >>                 <value>65536</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >> hdfs-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>dfs.block.size</name>
>>> >> >>                 <value>134217728</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>> >> >>                 <value>4096</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.support.append</name>
>>> >> >>                 <value>true</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.replication</name>
>>> >> >>                 <value>2</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >>
>>> >> >> Hope you can help us.
>>> >> >> Thanks in advance.
>>> >> >>
>>> >> >>
>>> >> >>
>>> >> >> --
>>> >> >> Davey Yan
>>> >> >
>>> >> >
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Davey Yan
>>> >
>>> >
>>>
>>>
>>>
>>> --
>>> Davey Yan
>>
>>
>
>
>
> --
> Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Hi Davey,

So were you able to find the issue?

JM

2013/2/25 Davey Yan <da...@gmail.com>:
> Hi Nicolas,
>
> I think i found what led to shutdown of all of the datanodes, but i am
> not completely certain.
> I will return to this mail list when my cluster returns to be stable.
>
> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>> Network error messages are not always friendly, especially if there is a
>> misconfiguration.
>> This said,  "connection refused" says that the network connection was made,
>> but that the remote port was not opened on the remote box. I.e. the process
>> was dead.
>> It could be useful to pastebin the whole logs as well...
>>
>>
>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>
>>> But... there was no log like "network unreachable".
>>>
>>>
>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>> wrote:
>>> > I agree.
>>> > Then for HDFS, ...
>>> > The first thing to check is the network I would say.
>>> >
>>> >
>>> >
>>> >
>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>> >>
>>> >> Thanks for reply, Nicolas.
>>> >>
>>> >> My question: What can lead to shutdown of all of the datanodes?
>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>> >>
>>> >>
>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>> >> wrote:
>>> >> > Ok, what's your question?
>>> >> > When you say the datanode went down, was it the datanode processes or
>>> >> > the
>>> >> > machines, with both the datanodes and the regionservers?
>>> >> >
>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>> >> > internally
>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>> >> > you
>>> >> > have
>>> >> > 'no answer for x minutes').
>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>> >> > considered
>>> >> > as live.
>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>> >> > (i.e.
>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>> >> > a
>>> >> > high
>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>> >> > refused
>>> >> > &
>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>> >> > does
>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>> >> > obviously,
>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>> >> > datanodes,
>>> >> > called "stale": an intermediary state where the datanode is used only
>>> >> > if
>>> >> > you
>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>> >> > It
>>> >> > will
>>> >> > be documented in HBase for the 0.96 release. But if all your
>>> >> > datanodes
>>> >> > are
>>> >> > down it won't change much.
>>> >> >
>>> >> > Cheers,
>>> >> >
>>> >> > Nicolas
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>> >> > wrote:
>>> >> >>
>>> >> >> Hey guys,
>>> >> >>
>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>> >> >> 1
>>> >> >> year, and it works fine.
>>> >> >> But the datanodes got shutdown twice in the last month.
>>> >> >>
>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>> >> >> but regionservers of HBase were still live in the HBase web
>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>> >> >> All of the processes of jvm were still running, including
>>> >> >> hmaster/namenode/regionserver/datanode.
>>> >> >>
>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>> >> >> of
>>> >> >> slaves became very high, more than 10, higher than normal running.
>>> >> >> From the "top" command, we saw that the processes of datanode and
>>> >> >> regionserver were comsuming CPU.
>>> >> >>
>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>> >> >> load of slaves returned to normal level, and we start the cluster
>>> >> >> again.
>>> >> >>
>>> >> >>
>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.148:50010
>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>> >> >>
>>> >> >>
>>> >> >> Logs in DNs indicated there were many IOException and
>>> >> >> SocketTimeoutException:
>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.io.IOException: Interrupted receiveBlock
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:03:44,823 WARN
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:48654]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:37188]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:12:41,892 INFO
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>> >> >> Verification
>>> >> >> succeeded for blk_-2674357249542194287_43419
>>> >> >>
>>> >> >>
>>> >> >> Here is our env:
>>> >> >> hadoop 1.0.3
>>> >> >> hbase 0.94.1(snappy enabled)
>>> >> >>
>>> >> >> java version "1.6.0_31"
>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>> >> >>
>>> >> >> # ulimit -a
>>> >> >> core file size          (blocks, -c) 0
>>> >> >> data seg size           (kbytes, -d) unlimited
>>> >> >> scheduling priority             (-e) 20
>>> >> >> file size               (blocks, -f) unlimited
>>> >> >> pending signals                 (-i) 16382
>>> >> >> max locked memory       (kbytes, -l) 64
>>> >> >> max memory size         (kbytes, -m) unlimited
>>> >> >> open files                      (-n) 32768
>>> >> >> pipe size            (512 bytes, -p) 8
>>> >> >> POSIX message queues     (bytes, -q) 819200
>>> >> >> real-time priority              (-r) 0
>>> >> >> stack size              (kbytes, -s) 8192
>>> >> >> cpu time               (seconds, -t) unlimited
>>> >> >> max user processes              (-u) 32768
>>> >> >> virtual memory          (kbytes, -v) unlimited
>>> >> >> file locks                      (-x) unlimited
>>> >> >>
>>> >> >> # uname -a
>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>> >> >> UTC 2011 x86_64 GNU/Linux
>>> >> >>
>>> >> >>
>>> >> >> # free(master)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>> >> >> 2580356
>>> >> >> -/+ buffers/cache:    5312768   19420168
>>> >> >> Swap:     72458232          0   72458232
>>> >> >>
>>> >> >>
>>> >> >> # free(slaves)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>> >> >> 15303304
>>> >> >> -/+ buffers/cache:    6658416   18074584
>>> >> >> Swap:     72458232        264   72457968
>>> >> >>
>>> >> >>
>>> >> >> Some important conf:
>>> >> >> core-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>io.file.buffer.size</name>
>>> >> >>                 <value>65536</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >> hdfs-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>dfs.block.size</name>
>>> >> >>                 <value>134217728</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>> >> >>                 <value>4096</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.support.append</name>
>>> >> >>                 <value>true</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.replication</name>
>>> >> >>                 <value>2</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >>
>>> >> >> Hope you can help us.
>>> >> >> Thanks in advance.
>>> >> >>
>>> >> >>
>>> >> >>
>>> >> >> --
>>> >> >> Davey Yan
>>> >> >
>>> >> >
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Davey Yan
>>> >
>>> >
>>>
>>>
>>>
>>> --
>>> Davey Yan
>>
>>
>
>
>
> --
> Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Hi Davey,

So were you able to find the issue?

JM

2013/2/25 Davey Yan <da...@gmail.com>:
> Hi Nicolas,
>
> I think i found what led to shutdown of all of the datanodes, but i am
> not completely certain.
> I will return to this mail list when my cluster returns to be stable.
>
> On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
>> Network error messages are not always friendly, especially if there is a
>> misconfiguration.
>> This said,  "connection refused" says that the network connection was made,
>> but that the remote port was not opened on the remote box. I.e. the process
>> was dead.
>> It could be useful to pastebin the whole logs as well...
>>
>>
>> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>>
>>> But... there was no log like "network unreachable".
>>>
>>>
>>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>>> wrote:
>>> > I agree.
>>> > Then for HDFS, ...
>>> > The first thing to check is the network I would say.
>>> >
>>> >
>>> >
>>> >
>>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>> >>
>>> >> Thanks for reply, Nicolas.
>>> >>
>>> >> My question: What can lead to shutdown of all of the datanodes?
>>> >> I believe that the regionservers will be OK if the HDFS is OK.
>>> >>
>>> >>
>>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>>> >> wrote:
>>> >> > Ok, what's your question?
>>> >> > When you say the datanode went down, was it the datanode processes or
>>> >> > the
>>> >> > machines, with both the datanodes and the regionservers?
>>> >> >
>>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>>> >> > internally
>>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>>> >> > you
>>> >> > have
>>> >> > 'no answer for x minutes').
>>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>>> >> > considered as dead after 180s with no answer. Before, well, it's
>>> >> > considered
>>> >> > as live.
>>> >> > When you stop a regionserver, it tries to flush its data to the disk
>>> >> > (i.e.
>>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>>> >> > a
>>> >> > high
>>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>>> >> > refused
>>> >> > &
>>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>>> >> > does
>>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>>> >> > obviously,
>>> >> > fails). Note that there is now  an intermediate state for hdfs
>>> >> > datanodes,
>>> >> > called "stale": an intermediary state where the datanode is used only
>>> >> > if
>>> >> > you
>>> >> > have to (i.e. it's the only datanode with a block replica you need).
>>> >> > It
>>> >> > will
>>> >> > be documented in HBase for the 0.96 release. But if all your
>>> >> > datanodes
>>> >> > are
>>> >> > down it won't change much.
>>> >> >
>>> >> > Cheers,
>>> >> >
>>> >> > Nicolas
>>> >> >
>>> >> >
>>> >> >
>>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>>> >> > wrote:
>>> >> >>
>>> >> >> Hey guys,
>>> >> >>
>>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>>> >> >> 1
>>> >> >> year, and it works fine.
>>> >> >> But the datanodes got shutdown twice in the last month.
>>> >> >>
>>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>>> >> >> but regionservers of HBase were still live in the HBase web
>>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>>> >> >> All of the processes of jvm were still running, including
>>> >> >> hmaster/namenode/regionserver/datanode.
>>> >> >>
>>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>>> >> >> of
>>> >> >> slaves became very high, more than 10, higher than normal running.
>>> >> >> From the "top" command, we saw that the processes of datanode and
>>> >> >> regionserver were comsuming CPU.
>>> >> >>
>>> >> >> We could not stop the HBase or Hadoop cluster through normal
>>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>>> >> >> load of slaves returned to normal level, and we start the cluster
>>> >> >> again.
>>> >> >>
>>> >> >>
>>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>>> >> >> 192.168.1.148:50010
>>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>>> >> >>
>>> >> >>
>>> >> >> Logs in DNs indicated there were many IOException and
>>> >> >> SocketTimeoutException:
>>> >> >> 2013-02-22 11:02:52,354 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.io.IOException: Interrupted receiveBlock
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:03:44,823 WARN
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:48654]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:09:42,294 ERROR
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>>> >> >> DatanodeRegistration(192.168.1.148:50010,
>>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>>> >> >> for channel to be ready for write. ch :
>>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>>> >> >> remote=/192.168.1.148:37188]
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>> >> >>         at
>>> >> >>
>>> >> >>
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>> >> >>         at java.lang.Thread.run(Thread.java:662)
>>> >> >> 2013-02-22 11:12:41,892 INFO
>>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>>> >> >> Verification
>>> >> >> succeeded for blk_-2674357249542194287_43419
>>> >> >>
>>> >> >>
>>> >> >> Here is our env:
>>> >> >> hadoop 1.0.3
>>> >> >> hbase 0.94.1(snappy enabled)
>>> >> >>
>>> >> >> java version "1.6.0_31"
>>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>> >> >>
>>> >> >> # ulimit -a
>>> >> >> core file size          (blocks, -c) 0
>>> >> >> data seg size           (kbytes, -d) unlimited
>>> >> >> scheduling priority             (-e) 20
>>> >> >> file size               (blocks, -f) unlimited
>>> >> >> pending signals                 (-i) 16382
>>> >> >> max locked memory       (kbytes, -l) 64
>>> >> >> max memory size         (kbytes, -m) unlimited
>>> >> >> open files                      (-n) 32768
>>> >> >> pipe size            (512 bytes, -p) 8
>>> >> >> POSIX message queues     (bytes, -q) 819200
>>> >> >> real-time priority              (-r) 0
>>> >> >> stack size              (kbytes, -s) 8192
>>> >> >> cpu time               (seconds, -t) unlimited
>>> >> >> max user processes              (-u) 32768
>>> >> >> virtual memory          (kbytes, -v) unlimited
>>> >> >> file locks                      (-x) unlimited
>>> >> >>
>>> >> >> # uname -a
>>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>>> >> >> UTC 2011 x86_64 GNU/Linux
>>> >> >>
>>> >> >>
>>> >> >> # free(master)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24732936    8383708   16349228          0     490584
>>> >> >> 2580356
>>> >> >> -/+ buffers/cache:    5312768   19420168
>>> >> >> Swap:     72458232          0   72458232
>>> >> >>
>>> >> >>
>>> >> >> # free(slaves)
>>> >> >>              total       used       free     shared    buffers
>>> >> >> cached
>>> >> >> Mem:      24733000   22824276    1908724          0     862556
>>> >> >> 15303304
>>> >> >> -/+ buffers/cache:    6658416   18074584
>>> >> >> Swap:     72458232        264   72457968
>>> >> >>
>>> >> >>
>>> >> >> Some important conf:
>>> >> >> core-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>io.file.buffer.size</name>
>>> >> >>                 <value>65536</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >> hdfs-site.xml
>>> >> >>         <property>
>>> >> >>                 <name>dfs.block.size</name>
>>> >> >>                 <value>134217728</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>>> >> >>                 <value>4096</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.support.append</name>
>>> >> >>                 <value>true</value>
>>> >> >>         </property>
>>> >> >>         <property>
>>> >> >>                 <name>dfs.replication</name>
>>> >> >>                 <value>2</value>
>>> >> >>         </property>
>>> >> >>
>>> >> >>
>>> >> >> Hope you can help us.
>>> >> >> Thanks in advance.
>>> >> >>
>>> >> >>
>>> >> >>
>>> >> >> --
>>> >> >> Davey Yan
>>> >> >
>>> >> >
>>> >>
>>> >>
>>> >>
>>> >> --
>>> >> Davey Yan
>>> >
>>> >
>>>
>>>
>>>
>>> --
>>> Davey Yan
>>
>>
>
>
>
> --
> Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Hi Nicolas,

I think i found what led to shutdown of all of the datanodes, but i am
not completely certain.
I will return to this mail list when my cluster returns to be stable.

On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Network error messages are not always friendly, especially if there is a
> misconfiguration.
> This said,  "connection refused" says that the network connection was made,
> but that the remote port was not opened on the remote box. I.e. the process
> was dead.
> It could be useful to pastebin the whole logs as well...
>
>
> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>
>> But... there was no log like "network unreachable".
>>
>>
>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > I agree.
>> > Then for HDFS, ...
>> > The first thing to check is the network I would say.
>> >
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Thanks for reply, Nicolas.
>> >>
>> >> My question: What can lead to shutdown of all of the datanodes?
>> >> I believe that the regionservers will be OK if the HDFS is OK.
>> >>
>> >>
>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> >> wrote:
>> >> > Ok, what's your question?
>> >> > When you say the datanode went down, was it the datanode processes or
>> >> > the
>> >> > machines, with both the datanodes and the regionservers?
>> >> >
>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>> >> > internally
>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>> >> > you
>> >> > have
>> >> > 'no answer for x minutes').
>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> >> > considered as dead after 180s with no answer. Before, well, it's
>> >> > considered
>> >> > as live.
>> >> > When you stop a regionserver, it tries to flush its data to the disk
>> >> > (i.e.
>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>> >> > a
>> >> > high
>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>> >> > refused
>> >> > &
>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> >> > does
>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>> >> > obviously,
>> >> > fails). Note that there is now  an intermediate state for hdfs
>> >> > datanodes,
>> >> > called "stale": an intermediary state where the datanode is used only
>> >> > if
>> >> > you
>> >> > have to (i.e. it's the only datanode with a block replica you need).
>> >> > It
>> >> > will
>> >> > be documented in HBase for the 0.96 release. But if all your
>> >> > datanodes
>> >> > are
>> >> > down it won't change much.
>> >> >
>> >> > Cheers,
>> >> >
>> >> > Nicolas
>> >> >
>> >> >
>> >> >
>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Hey guys,
>> >> >>
>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>> >> >> 1
>> >> >> year, and it works fine.
>> >> >> But the datanodes got shutdown twice in the last month.
>> >> >>
>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> >> but regionservers of HBase were still live in the HBase web
>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> >> All of the processes of jvm were still running, including
>> >> >> hmaster/namenode/regionserver/datanode.
>> >> >>
>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>> >> >> of
>> >> >> slaves became very high, more than 10, higher than normal running.
>> >> >> From the "top" command, we saw that the processes of datanode and
>> >> >> regionserver were comsuming CPU.
>> >> >>
>> >> >> We could not stop the HBase or Hadoop cluster through normal
>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> >> load of slaves returned to normal level, and we start the cluster
>> >> >> again.
>> >> >>
>> >> >>
>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.148:50010
>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>> >> >>
>> >> >>
>> >> >> Logs in DNs indicated there were many IOException and
>> >> >> SocketTimeoutException:
>> >> >> 2013-02-22 11:02:52,354 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.io.IOException: Interrupted receiveBlock
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:03:44,823 WARN
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:48654]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:09:42,294 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:37188]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:12:41,892 INFO
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>> >> >> Verification
>> >> >> succeeded for blk_-2674357249542194287_43419
>> >> >>
>> >> >>
>> >> >> Here is our env:
>> >> >> hadoop 1.0.3
>> >> >> hbase 0.94.1(snappy enabled)
>> >> >>
>> >> >> java version "1.6.0_31"
>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >> >>
>> >> >> # ulimit -a
>> >> >> core file size          (blocks, -c) 0
>> >> >> data seg size           (kbytes, -d) unlimited
>> >> >> scheduling priority             (-e) 20
>> >> >> file size               (blocks, -f) unlimited
>> >> >> pending signals                 (-i) 16382
>> >> >> max locked memory       (kbytes, -l) 64
>> >> >> max memory size         (kbytes, -m) unlimited
>> >> >> open files                      (-n) 32768
>> >> >> pipe size            (512 bytes, -p) 8
>> >> >> POSIX message queues     (bytes, -q) 819200
>> >> >> real-time priority              (-r) 0
>> >> >> stack size              (kbytes, -s) 8192
>> >> >> cpu time               (seconds, -t) unlimited
>> >> >> max user processes              (-u) 32768
>> >> >> virtual memory          (kbytes, -v) unlimited
>> >> >> file locks                      (-x) unlimited
>> >> >>
>> >> >> # uname -a
>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> >> UTC 2011 x86_64 GNU/Linux
>> >> >>
>> >> >>
>> >> >> # free(master)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24732936    8383708   16349228          0     490584
>> >> >> 2580356
>> >> >> -/+ buffers/cache:    5312768   19420168
>> >> >> Swap:     72458232          0   72458232
>> >> >>
>> >> >>
>> >> >> # free(slaves)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24733000   22824276    1908724          0     862556
>> >> >> 15303304
>> >> >> -/+ buffers/cache:    6658416   18074584
>> >> >> Swap:     72458232        264   72457968
>> >> >>
>> >> >>
>> >> >> Some important conf:
>> >> >> core-site.xml
>> >> >>         <property>
>> >> >>                 <name>io.file.buffer.size</name>
>> >> >>                 <value>65536</value>
>> >> >>         </property>
>> >> >>
>> >> >> hdfs-site.xml
>> >> >>         <property>
>> >> >>                 <name>dfs.block.size</name>
>> >> >>                 <value>134217728</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>> >> >>                 <value>4096</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.support.append</name>
>> >> >>                 <value>true</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.replication</name>
>> >> >>                 <value>2</value>
>> >> >>         </property>
>> >> >>
>> >> >>
>> >> >> Hope you can help us.
>> >> >> Thanks in advance.
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Davey Yan
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Hi Nicolas,

I think i found what led to shutdown of all of the datanodes, but i am
not completely certain.
I will return to this mail list when my cluster returns to be stable.

On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Network error messages are not always friendly, especially if there is a
> misconfiguration.
> This said,  "connection refused" says that the network connection was made,
> but that the remote port was not opened on the remote box. I.e. the process
> was dead.
> It could be useful to pastebin the whole logs as well...
>
>
> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>
>> But... there was no log like "network unreachable".
>>
>>
>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > I agree.
>> > Then for HDFS, ...
>> > The first thing to check is the network I would say.
>> >
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Thanks for reply, Nicolas.
>> >>
>> >> My question: What can lead to shutdown of all of the datanodes?
>> >> I believe that the regionservers will be OK if the HDFS is OK.
>> >>
>> >>
>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> >> wrote:
>> >> > Ok, what's your question?
>> >> > When you say the datanode went down, was it the datanode processes or
>> >> > the
>> >> > machines, with both the datanodes and the regionservers?
>> >> >
>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>> >> > internally
>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>> >> > you
>> >> > have
>> >> > 'no answer for x minutes').
>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> >> > considered as dead after 180s with no answer. Before, well, it's
>> >> > considered
>> >> > as live.
>> >> > When you stop a regionserver, it tries to flush its data to the disk
>> >> > (i.e.
>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>> >> > a
>> >> > high
>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>> >> > refused
>> >> > &
>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> >> > does
>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>> >> > obviously,
>> >> > fails). Note that there is now  an intermediate state for hdfs
>> >> > datanodes,
>> >> > called "stale": an intermediary state where the datanode is used only
>> >> > if
>> >> > you
>> >> > have to (i.e. it's the only datanode with a block replica you need).
>> >> > It
>> >> > will
>> >> > be documented in HBase for the 0.96 release. But if all your
>> >> > datanodes
>> >> > are
>> >> > down it won't change much.
>> >> >
>> >> > Cheers,
>> >> >
>> >> > Nicolas
>> >> >
>> >> >
>> >> >
>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Hey guys,
>> >> >>
>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>> >> >> 1
>> >> >> year, and it works fine.
>> >> >> But the datanodes got shutdown twice in the last month.
>> >> >>
>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> >> but regionservers of HBase were still live in the HBase web
>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> >> All of the processes of jvm were still running, including
>> >> >> hmaster/namenode/regionserver/datanode.
>> >> >>
>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>> >> >> of
>> >> >> slaves became very high, more than 10, higher than normal running.
>> >> >> From the "top" command, we saw that the processes of datanode and
>> >> >> regionserver were comsuming CPU.
>> >> >>
>> >> >> We could not stop the HBase or Hadoop cluster through normal
>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> >> load of slaves returned to normal level, and we start the cluster
>> >> >> again.
>> >> >>
>> >> >>
>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.148:50010
>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>> >> >>
>> >> >>
>> >> >> Logs in DNs indicated there were many IOException and
>> >> >> SocketTimeoutException:
>> >> >> 2013-02-22 11:02:52,354 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.io.IOException: Interrupted receiveBlock
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:03:44,823 WARN
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:48654]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:09:42,294 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:37188]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:12:41,892 INFO
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>> >> >> Verification
>> >> >> succeeded for blk_-2674357249542194287_43419
>> >> >>
>> >> >>
>> >> >> Here is our env:
>> >> >> hadoop 1.0.3
>> >> >> hbase 0.94.1(snappy enabled)
>> >> >>
>> >> >> java version "1.6.0_31"
>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >> >>
>> >> >> # ulimit -a
>> >> >> core file size          (blocks, -c) 0
>> >> >> data seg size           (kbytes, -d) unlimited
>> >> >> scheduling priority             (-e) 20
>> >> >> file size               (blocks, -f) unlimited
>> >> >> pending signals                 (-i) 16382
>> >> >> max locked memory       (kbytes, -l) 64
>> >> >> max memory size         (kbytes, -m) unlimited
>> >> >> open files                      (-n) 32768
>> >> >> pipe size            (512 bytes, -p) 8
>> >> >> POSIX message queues     (bytes, -q) 819200
>> >> >> real-time priority              (-r) 0
>> >> >> stack size              (kbytes, -s) 8192
>> >> >> cpu time               (seconds, -t) unlimited
>> >> >> max user processes              (-u) 32768
>> >> >> virtual memory          (kbytes, -v) unlimited
>> >> >> file locks                      (-x) unlimited
>> >> >>
>> >> >> # uname -a
>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> >> UTC 2011 x86_64 GNU/Linux
>> >> >>
>> >> >>
>> >> >> # free(master)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24732936    8383708   16349228          0     490584
>> >> >> 2580356
>> >> >> -/+ buffers/cache:    5312768   19420168
>> >> >> Swap:     72458232          0   72458232
>> >> >>
>> >> >>
>> >> >> # free(slaves)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24733000   22824276    1908724          0     862556
>> >> >> 15303304
>> >> >> -/+ buffers/cache:    6658416   18074584
>> >> >> Swap:     72458232        264   72457968
>> >> >>
>> >> >>
>> >> >> Some important conf:
>> >> >> core-site.xml
>> >> >>         <property>
>> >> >>                 <name>io.file.buffer.size</name>
>> >> >>                 <value>65536</value>
>> >> >>         </property>
>> >> >>
>> >> >> hdfs-site.xml
>> >> >>         <property>
>> >> >>                 <name>dfs.block.size</name>
>> >> >>                 <value>134217728</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>> >> >>                 <value>4096</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.support.append</name>
>> >> >>                 <value>true</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.replication</name>
>> >> >>                 <value>2</value>
>> >> >>         </property>
>> >> >>
>> >> >>
>> >> >> Hope you can help us.
>> >> >> Thanks in advance.
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Davey Yan
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Hi Nicolas,

I think i found what led to shutdown of all of the datanodes, but i am
not completely certain.
I will return to this mail list when my cluster returns to be stable.

On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Network error messages are not always friendly, especially if there is a
> misconfiguration.
> This said,  "connection refused" says that the network connection was made,
> but that the remote port was not opened on the remote box. I.e. the process
> was dead.
> It could be useful to pastebin the whole logs as well...
>
>
> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>
>> But... there was no log like "network unreachable".
>>
>>
>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > I agree.
>> > Then for HDFS, ...
>> > The first thing to check is the network I would say.
>> >
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Thanks for reply, Nicolas.
>> >>
>> >> My question: What can lead to shutdown of all of the datanodes?
>> >> I believe that the regionservers will be OK if the HDFS is OK.
>> >>
>> >>
>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> >> wrote:
>> >> > Ok, what's your question?
>> >> > When you say the datanode went down, was it the datanode processes or
>> >> > the
>> >> > machines, with both the datanodes and the regionservers?
>> >> >
>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>> >> > internally
>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>> >> > you
>> >> > have
>> >> > 'no answer for x minutes').
>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> >> > considered as dead after 180s with no answer. Before, well, it's
>> >> > considered
>> >> > as live.
>> >> > When you stop a regionserver, it tries to flush its data to the disk
>> >> > (i.e.
>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>> >> > a
>> >> > high
>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>> >> > refused
>> >> > &
>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> >> > does
>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>> >> > obviously,
>> >> > fails). Note that there is now  an intermediate state for hdfs
>> >> > datanodes,
>> >> > called "stale": an intermediary state where the datanode is used only
>> >> > if
>> >> > you
>> >> > have to (i.e. it's the only datanode with a block replica you need).
>> >> > It
>> >> > will
>> >> > be documented in HBase for the 0.96 release. But if all your
>> >> > datanodes
>> >> > are
>> >> > down it won't change much.
>> >> >
>> >> > Cheers,
>> >> >
>> >> > Nicolas
>> >> >
>> >> >
>> >> >
>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Hey guys,
>> >> >>
>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>> >> >> 1
>> >> >> year, and it works fine.
>> >> >> But the datanodes got shutdown twice in the last month.
>> >> >>
>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> >> but regionservers of HBase were still live in the HBase web
>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> >> All of the processes of jvm were still running, including
>> >> >> hmaster/namenode/regionserver/datanode.
>> >> >>
>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>> >> >> of
>> >> >> slaves became very high, more than 10, higher than normal running.
>> >> >> From the "top" command, we saw that the processes of datanode and
>> >> >> regionserver were comsuming CPU.
>> >> >>
>> >> >> We could not stop the HBase or Hadoop cluster through normal
>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> >> load of slaves returned to normal level, and we start the cluster
>> >> >> again.
>> >> >>
>> >> >>
>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.148:50010
>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>> >> >>
>> >> >>
>> >> >> Logs in DNs indicated there were many IOException and
>> >> >> SocketTimeoutException:
>> >> >> 2013-02-22 11:02:52,354 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.io.IOException: Interrupted receiveBlock
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:03:44,823 WARN
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:48654]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:09:42,294 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:37188]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:12:41,892 INFO
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>> >> >> Verification
>> >> >> succeeded for blk_-2674357249542194287_43419
>> >> >>
>> >> >>
>> >> >> Here is our env:
>> >> >> hadoop 1.0.3
>> >> >> hbase 0.94.1(snappy enabled)
>> >> >>
>> >> >> java version "1.6.0_31"
>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >> >>
>> >> >> # ulimit -a
>> >> >> core file size          (blocks, -c) 0
>> >> >> data seg size           (kbytes, -d) unlimited
>> >> >> scheduling priority             (-e) 20
>> >> >> file size               (blocks, -f) unlimited
>> >> >> pending signals                 (-i) 16382
>> >> >> max locked memory       (kbytes, -l) 64
>> >> >> max memory size         (kbytes, -m) unlimited
>> >> >> open files                      (-n) 32768
>> >> >> pipe size            (512 bytes, -p) 8
>> >> >> POSIX message queues     (bytes, -q) 819200
>> >> >> real-time priority              (-r) 0
>> >> >> stack size              (kbytes, -s) 8192
>> >> >> cpu time               (seconds, -t) unlimited
>> >> >> max user processes              (-u) 32768
>> >> >> virtual memory          (kbytes, -v) unlimited
>> >> >> file locks                      (-x) unlimited
>> >> >>
>> >> >> # uname -a
>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> >> UTC 2011 x86_64 GNU/Linux
>> >> >>
>> >> >>
>> >> >> # free(master)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24732936    8383708   16349228          0     490584
>> >> >> 2580356
>> >> >> -/+ buffers/cache:    5312768   19420168
>> >> >> Swap:     72458232          0   72458232
>> >> >>
>> >> >>
>> >> >> # free(slaves)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24733000   22824276    1908724          0     862556
>> >> >> 15303304
>> >> >> -/+ buffers/cache:    6658416   18074584
>> >> >> Swap:     72458232        264   72457968
>> >> >>
>> >> >>
>> >> >> Some important conf:
>> >> >> core-site.xml
>> >> >>         <property>
>> >> >>                 <name>io.file.buffer.size</name>
>> >> >>                 <value>65536</value>
>> >> >>         </property>
>> >> >>
>> >> >> hdfs-site.xml
>> >> >>         <property>
>> >> >>                 <name>dfs.block.size</name>
>> >> >>                 <value>134217728</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>> >> >>                 <value>4096</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.support.append</name>
>> >> >>                 <value>true</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.replication</name>
>> >> >>                 <value>2</value>
>> >> >>         </property>
>> >> >>
>> >> >>
>> >> >> Hope you can help us.
>> >> >> Thanks in advance.
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Davey Yan
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Hi Nicolas,

I think i found what led to shutdown of all of the datanodes, but i am
not completely certain.
I will return to this mail list when my cluster returns to be stable.

On Mon, Feb 25, 2013 at 8:01 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Network error messages are not always friendly, especially if there is a
> misconfiguration.
> This said,  "connection refused" says that the network connection was made,
> but that the remote port was not opened on the remote box. I.e. the process
> was dead.
> It could be useful to pastebin the whole logs as well...
>
>
> On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:
>>
>> But... there was no log like "network unreachable".
>>
>>
>> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > I agree.
>> > Then for HDFS, ...
>> > The first thing to check is the network I would say.
>> >
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Thanks for reply, Nicolas.
>> >>
>> >> My question: What can lead to shutdown of all of the datanodes?
>> >> I believe that the regionservers will be OK if the HDFS is OK.
>> >>
>> >>
>> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> >> wrote:
>> >> > Ok, what's your question?
>> >> > When you say the datanode went down, was it the datanode processes or
>> >> > the
>> >> > machines, with both the datanodes and the regionservers?
>> >> >
>> >> > The NameNode pings its datanodes every 3 seconds. However it will
>> >> > internally
>> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui
>> >> > you
>> >> > have
>> >> > 'no answer for x minutes').
>> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> >> > considered as dead after 180s with no answer. Before, well, it's
>> >> > considered
>> >> > as live.
>> >> > When you stop a regionserver, it tries to flush its data to the disk
>> >> > (i.e.
>> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
>> >> > a
>> >> > high
>> >> > ratio of your datanodes are dead, it can't shutdown. Connection
>> >> > refused
>> >> > &
>> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> >> > does
>> >> > not declare the nodes as dead, so hbase tries to use them (and,
>> >> > obviously,
>> >> > fails). Note that there is now  an intermediate state for hdfs
>> >> > datanodes,
>> >> > called "stale": an intermediary state where the datanode is used only
>> >> > if
>> >> > you
>> >> > have to (i.e. it's the only datanode with a block replica you need).
>> >> > It
>> >> > will
>> >> > be documented in HBase for the 0.96 release. But if all your
>> >> > datanodes
>> >> > are
>> >> > down it won't change much.
>> >> >
>> >> > Cheers,
>> >> >
>> >> > Nicolas
>> >> >
>> >> >
>> >> >
>> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
>> >> > wrote:
>> >> >>
>> >> >> Hey guys,
>> >> >>
>> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
>> >> >> 1
>> >> >> year, and it works fine.
>> >> >> But the datanodes got shutdown twice in the last month.
>> >> >>
>> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> >> but regionservers of HBase were still live in the HBase web
>> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> >> All of the processes of jvm were still running, including
>> >> >> hmaster/namenode/regionserver/datanode.
>> >> >>
>> >> >> When the datanodes got shutdown, the load (using the "top" command)
>> >> >> of
>> >> >> slaves became very high, more than 10, higher than normal running.
>> >> >> From the "top" command, we saw that the processes of datanode and
>> >> >> regionserver were comsuming CPU.
>> >> >>
>> >> >> We could not stop the HBase or Hadoop cluster through normal
>> >> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> >> load of slaves returned to normal level, and we start the cluster
>> >> >> again.
>> >> >>
>> >> >>
>> >> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> >> 192.168.1.148:50010
>> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> >> Removing a node: /default-rack/192.168.1.148:50010
>> >> >>
>> >> >>
>> >> >> Logs in DNs indicated there were many IOException and
>> >> >> SocketTimeoutException:
>> >> >> 2013-02-22 11:02:52,354 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.io.IOException: Interrupted receiveBlock
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:03:44,823 WARN
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:48654]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:09:42,294 ERROR
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> >> DatanodeRegistration(192.168.1.148:50010,
>> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> >> for channel to be ready for write. ch :
>> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> >> remote=/192.168.1.148:37188]
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >> >>         at
>> >> >>
>> >> >>
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >> >>         at java.lang.Thread.run(Thread.java:662)
>> >> >> 2013-02-22 11:12:41,892 INFO
>> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner:
>> >> >> Verification
>> >> >> succeeded for blk_-2674357249542194287_43419
>> >> >>
>> >> >>
>> >> >> Here is our env:
>> >> >> hadoop 1.0.3
>> >> >> hbase 0.94.1(snappy enabled)
>> >> >>
>> >> >> java version "1.6.0_31"
>> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >> >>
>> >> >> # ulimit -a
>> >> >> core file size          (blocks, -c) 0
>> >> >> data seg size           (kbytes, -d) unlimited
>> >> >> scheduling priority             (-e) 20
>> >> >> file size               (blocks, -f) unlimited
>> >> >> pending signals                 (-i) 16382
>> >> >> max locked memory       (kbytes, -l) 64
>> >> >> max memory size         (kbytes, -m) unlimited
>> >> >> open files                      (-n) 32768
>> >> >> pipe size            (512 bytes, -p) 8
>> >> >> POSIX message queues     (bytes, -q) 819200
>> >> >> real-time priority              (-r) 0
>> >> >> stack size              (kbytes, -s) 8192
>> >> >> cpu time               (seconds, -t) unlimited
>> >> >> max user processes              (-u) 32768
>> >> >> virtual memory          (kbytes, -v) unlimited
>> >> >> file locks                      (-x) unlimited
>> >> >>
>> >> >> # uname -a
>> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> >> UTC 2011 x86_64 GNU/Linux
>> >> >>
>> >> >>
>> >> >> # free(master)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24732936    8383708   16349228          0     490584
>> >> >> 2580356
>> >> >> -/+ buffers/cache:    5312768   19420168
>> >> >> Swap:     72458232          0   72458232
>> >> >>
>> >> >>
>> >> >> # free(slaves)
>> >> >>              total       used       free     shared    buffers
>> >> >> cached
>> >> >> Mem:      24733000   22824276    1908724          0     862556
>> >> >> 15303304
>> >> >> -/+ buffers/cache:    6658416   18074584
>> >> >> Swap:     72458232        264   72457968
>> >> >>
>> >> >>
>> >> >> Some important conf:
>> >> >> core-site.xml
>> >> >>         <property>
>> >> >>                 <name>io.file.buffer.size</name>
>> >> >>                 <value>65536</value>
>> >> >>         </property>
>> >> >>
>> >> >> hdfs-site.xml
>> >> >>         <property>
>> >> >>                 <name>dfs.block.size</name>
>> >> >>                 <value>134217728</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.datanode.max.xcievers</name>
>> >> >>                 <value>4096</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.support.append</name>
>> >> >>                 <value>true</value>
>> >> >>         </property>
>> >> >>         <property>
>> >> >>                 <name>dfs.replication</name>
>> >> >>                 <value>2</value>
>> >> >>         </property>
>> >> >>
>> >> >>
>> >> >> Hope you can help us.
>> >> >> Thanks in advance.
>> >> >>
>> >> >>
>> >> >>
>> >> >> --
>> >> >> Davey Yan
>> >> >
>> >> >
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Network error messages are not always friendly, especially if there is a
misconfiguration.
This said,  "connection refused" says that the network connection was made,
but that the remote port was not opened on the remote box. I.e. the process
was dead.
It could be useful to pastebin the whole logs as well...


On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:

> But... there was no log like "network unreachable".
>
>
> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > I agree.
> > Then for HDFS, ...
> > The first thing to check is the network I would say.
> >
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Thanks for reply, Nicolas.
> >>
> >> My question: What can lead to shutdown of all of the datanodes?
> >> I believe that the regionservers will be OK if the HDFS is OK.
> >>
> >>
> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >> > Ok, what's your question?
> >> > When you say the datanode went down, was it the datanode processes or
> >> > the
> >> > machines, with both the datanodes and the regionservers?
> >> >
> >> > The NameNode pings its datanodes every 3 seconds. However it will
> >> > internally
> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> >> > have
> >> > 'no answer for x minutes').
> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> >> > considered as dead after 180s with no answer. Before, well, it's
> >> > considered
> >> > as live.
> >> > When you stop a regionserver, it tries to flush its data to the disk
> >> > (i.e.
> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
> a
> >> > high
> >> > ratio of your datanodes are dead, it can't shutdown. Connection
> refused
> >> > &
> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> >> > does
> >> > not declare the nodes as dead, so hbase tries to use them (and,
> >> > obviously,
> >> > fails). Note that there is now  an intermediate state for hdfs
> >> > datanodes,
> >> > called "stale": an intermediary state where the datanode is used only
> if
> >> > you
> >> > have to (i.e. it's the only datanode with a block replica you need).
> It
> >> > will
> >> > be documented in HBase for the 0.96 release. But if all your datanodes
> >> > are
> >> > down it won't change much.
> >> >
> >> > Cheers,
> >> >
> >> > Nicolas
> >> >
> >> >
> >> >
> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
> wrote:
> >> >>
> >> >> Hey guys,
> >> >>
> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
> 1
> >> >> year, and it works fine.
> >> >> But the datanodes got shutdown twice in the last month.
> >> >>
> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> >> but regionservers of HBase were still live in the HBase web
> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> >> All of the processes of jvm were still running, including
> >> >> hmaster/namenode/regionserver/datanode.
> >> >>
> >> >> When the datanodes got shutdown, the load (using the "top" command)
> of
> >> >> slaves became very high, more than 10, higher than normal running.
> >> >> From the "top" command, we saw that the processes of datanode and
> >> >> regionserver were comsuming CPU.
> >> >>
> >> >> We could not stop the HBase or Hadoop cluster through normal
> >> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> >> load of slaves returned to normal level, and we start the cluster
> >> >> again.
> >> >>
> >> >>
> >> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.152:50010
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.148:50010
> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.148:50010
> >> >>
> >> >>
> >> >> Logs in DNs indicated there were many IOException and
> >> >> SocketTimeoutException:
> >> >> 2013-02-22 11:02:52,354 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.io.IOException: Interrupted receiveBlock
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:03:44,823 WARN
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:48654]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:09:42,294 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:37188]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:12:41,892 INFO
> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> >> succeeded for blk_-2674357249542194287_43419
> >> >>
> >> >>
> >> >> Here is our env:
> >> >> hadoop 1.0.3
> >> >> hbase 0.94.1(snappy enabled)
> >> >>
> >> >> java version "1.6.0_31"
> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >> >>
> >> >> # ulimit -a
> >> >> core file size          (blocks, -c) 0
> >> >> data seg size           (kbytes, -d) unlimited
> >> >> scheduling priority             (-e) 20
> >> >> file size               (blocks, -f) unlimited
> >> >> pending signals                 (-i) 16382
> >> >> max locked memory       (kbytes, -l) 64
> >> >> max memory size         (kbytes, -m) unlimited
> >> >> open files                      (-n) 32768
> >> >> pipe size            (512 bytes, -p) 8
> >> >> POSIX message queues     (bytes, -q) 819200
> >> >> real-time priority              (-r) 0
> >> >> stack size              (kbytes, -s) 8192
> >> >> cpu time               (seconds, -t) unlimited
> >> >> max user processes              (-u) 32768
> >> >> virtual memory          (kbytes, -v) unlimited
> >> >> file locks                      (-x) unlimited
> >> >>
> >> >> # uname -a
> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> >> UTC 2011 x86_64 GNU/Linux
> >> >>
> >> >>
> >> >> # free(master)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24732936    8383708   16349228          0     490584
> >> >> 2580356
> >> >> -/+ buffers/cache:    5312768   19420168
> >> >> Swap:     72458232          0   72458232
> >> >>
> >> >>
> >> >> # free(slaves)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24733000   22824276    1908724          0     862556
> >> >> 15303304
> >> >> -/+ buffers/cache:    6658416   18074584
> >> >> Swap:     72458232        264   72457968
> >> >>
> >> >>
> >> >> Some important conf:
> >> >> core-site.xml
> >> >>         <property>
> >> >>                 <name>io.file.buffer.size</name>
> >> >>                 <value>65536</value>
> >> >>         </property>
> >> >>
> >> >> hdfs-site.xml
> >> >>         <property>
> >> >>                 <name>dfs.block.size</name>
> >> >>                 <value>134217728</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.datanode.max.xcievers</name>
> >> >>                 <value>4096</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.support.append</name>
> >> >>                 <value>true</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.replication</name>
> >> >>                 <value>2</value>
> >> >>         </property>
> >> >>
> >> >>
> >> >> Hope you can help us.
> >> >> Thanks in advance.
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Davey Yan
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Network error messages are not always friendly, especially if there is a
misconfiguration.
This said,  "connection refused" says that the network connection was made,
but that the remote port was not opened on the remote box. I.e. the process
was dead.
It could be useful to pastebin the whole logs as well...


On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:

> But... there was no log like "network unreachable".
>
>
> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > I agree.
> > Then for HDFS, ...
> > The first thing to check is the network I would say.
> >
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Thanks for reply, Nicolas.
> >>
> >> My question: What can lead to shutdown of all of the datanodes?
> >> I believe that the regionservers will be OK if the HDFS is OK.
> >>
> >>
> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >> > Ok, what's your question?
> >> > When you say the datanode went down, was it the datanode processes or
> >> > the
> >> > machines, with both the datanodes and the regionservers?
> >> >
> >> > The NameNode pings its datanodes every 3 seconds. However it will
> >> > internally
> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> >> > have
> >> > 'no answer for x minutes').
> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> >> > considered as dead after 180s with no answer. Before, well, it's
> >> > considered
> >> > as live.
> >> > When you stop a regionserver, it tries to flush its data to the disk
> >> > (i.e.
> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
> a
> >> > high
> >> > ratio of your datanodes are dead, it can't shutdown. Connection
> refused
> >> > &
> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> >> > does
> >> > not declare the nodes as dead, so hbase tries to use them (and,
> >> > obviously,
> >> > fails). Note that there is now  an intermediate state for hdfs
> >> > datanodes,
> >> > called "stale": an intermediary state where the datanode is used only
> if
> >> > you
> >> > have to (i.e. it's the only datanode with a block replica you need).
> It
> >> > will
> >> > be documented in HBase for the 0.96 release. But if all your datanodes
> >> > are
> >> > down it won't change much.
> >> >
> >> > Cheers,
> >> >
> >> > Nicolas
> >> >
> >> >
> >> >
> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
> wrote:
> >> >>
> >> >> Hey guys,
> >> >>
> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
> 1
> >> >> year, and it works fine.
> >> >> But the datanodes got shutdown twice in the last month.
> >> >>
> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> >> but regionservers of HBase were still live in the HBase web
> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> >> All of the processes of jvm were still running, including
> >> >> hmaster/namenode/regionserver/datanode.
> >> >>
> >> >> When the datanodes got shutdown, the load (using the "top" command)
> of
> >> >> slaves became very high, more than 10, higher than normal running.
> >> >> From the "top" command, we saw that the processes of datanode and
> >> >> regionserver were comsuming CPU.
> >> >>
> >> >> We could not stop the HBase or Hadoop cluster through normal
> >> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> >> load of slaves returned to normal level, and we start the cluster
> >> >> again.
> >> >>
> >> >>
> >> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.152:50010
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.148:50010
> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.148:50010
> >> >>
> >> >>
> >> >> Logs in DNs indicated there were many IOException and
> >> >> SocketTimeoutException:
> >> >> 2013-02-22 11:02:52,354 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.io.IOException: Interrupted receiveBlock
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:03:44,823 WARN
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:48654]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:09:42,294 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:37188]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:12:41,892 INFO
> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> >> succeeded for blk_-2674357249542194287_43419
> >> >>
> >> >>
> >> >> Here is our env:
> >> >> hadoop 1.0.3
> >> >> hbase 0.94.1(snappy enabled)
> >> >>
> >> >> java version "1.6.0_31"
> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >> >>
> >> >> # ulimit -a
> >> >> core file size          (blocks, -c) 0
> >> >> data seg size           (kbytes, -d) unlimited
> >> >> scheduling priority             (-e) 20
> >> >> file size               (blocks, -f) unlimited
> >> >> pending signals                 (-i) 16382
> >> >> max locked memory       (kbytes, -l) 64
> >> >> max memory size         (kbytes, -m) unlimited
> >> >> open files                      (-n) 32768
> >> >> pipe size            (512 bytes, -p) 8
> >> >> POSIX message queues     (bytes, -q) 819200
> >> >> real-time priority              (-r) 0
> >> >> stack size              (kbytes, -s) 8192
> >> >> cpu time               (seconds, -t) unlimited
> >> >> max user processes              (-u) 32768
> >> >> virtual memory          (kbytes, -v) unlimited
> >> >> file locks                      (-x) unlimited
> >> >>
> >> >> # uname -a
> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> >> UTC 2011 x86_64 GNU/Linux
> >> >>
> >> >>
> >> >> # free(master)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24732936    8383708   16349228          0     490584
> >> >> 2580356
> >> >> -/+ buffers/cache:    5312768   19420168
> >> >> Swap:     72458232          0   72458232
> >> >>
> >> >>
> >> >> # free(slaves)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24733000   22824276    1908724          0     862556
> >> >> 15303304
> >> >> -/+ buffers/cache:    6658416   18074584
> >> >> Swap:     72458232        264   72457968
> >> >>
> >> >>
> >> >> Some important conf:
> >> >> core-site.xml
> >> >>         <property>
> >> >>                 <name>io.file.buffer.size</name>
> >> >>                 <value>65536</value>
> >> >>         </property>
> >> >>
> >> >> hdfs-site.xml
> >> >>         <property>
> >> >>                 <name>dfs.block.size</name>
> >> >>                 <value>134217728</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.datanode.max.xcievers</name>
> >> >>                 <value>4096</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.support.append</name>
> >> >>                 <value>true</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.replication</name>
> >> >>                 <value>2</value>
> >> >>         </property>
> >> >>
> >> >>
> >> >> Hope you can help us.
> >> >> Thanks in advance.
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Davey Yan
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Network error messages are not always friendly, especially if there is a
misconfiguration.
This said,  "connection refused" says that the network connection was made,
but that the remote port was not opened on the remote box. I.e. the process
was dead.
It could be useful to pastebin the whole logs as well...


On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:

> But... there was no log like "network unreachable".
>
>
> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > I agree.
> > Then for HDFS, ...
> > The first thing to check is the network I would say.
> >
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Thanks for reply, Nicolas.
> >>
> >> My question: What can lead to shutdown of all of the datanodes?
> >> I believe that the regionservers will be OK if the HDFS is OK.
> >>
> >>
> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >> > Ok, what's your question?
> >> > When you say the datanode went down, was it the datanode processes or
> >> > the
> >> > machines, with both the datanodes and the regionservers?
> >> >
> >> > The NameNode pings its datanodes every 3 seconds. However it will
> >> > internally
> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> >> > have
> >> > 'no answer for x minutes').
> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> >> > considered as dead after 180s with no answer. Before, well, it's
> >> > considered
> >> > as live.
> >> > When you stop a regionserver, it tries to flush its data to the disk
> >> > (i.e.
> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
> a
> >> > high
> >> > ratio of your datanodes are dead, it can't shutdown. Connection
> refused
> >> > &
> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> >> > does
> >> > not declare the nodes as dead, so hbase tries to use them (and,
> >> > obviously,
> >> > fails). Note that there is now  an intermediate state for hdfs
> >> > datanodes,
> >> > called "stale": an intermediary state where the datanode is used only
> if
> >> > you
> >> > have to (i.e. it's the only datanode with a block replica you need).
> It
> >> > will
> >> > be documented in HBase for the 0.96 release. But if all your datanodes
> >> > are
> >> > down it won't change much.
> >> >
> >> > Cheers,
> >> >
> >> > Nicolas
> >> >
> >> >
> >> >
> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
> wrote:
> >> >>
> >> >> Hey guys,
> >> >>
> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
> 1
> >> >> year, and it works fine.
> >> >> But the datanodes got shutdown twice in the last month.
> >> >>
> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> >> but regionservers of HBase were still live in the HBase web
> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> >> All of the processes of jvm were still running, including
> >> >> hmaster/namenode/regionserver/datanode.
> >> >>
> >> >> When the datanodes got shutdown, the load (using the "top" command)
> of
> >> >> slaves became very high, more than 10, higher than normal running.
> >> >> From the "top" command, we saw that the processes of datanode and
> >> >> regionserver were comsuming CPU.
> >> >>
> >> >> We could not stop the HBase or Hadoop cluster through normal
> >> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> >> load of slaves returned to normal level, and we start the cluster
> >> >> again.
> >> >>
> >> >>
> >> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.152:50010
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.148:50010
> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.148:50010
> >> >>
> >> >>
> >> >> Logs in DNs indicated there were many IOException and
> >> >> SocketTimeoutException:
> >> >> 2013-02-22 11:02:52,354 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.io.IOException: Interrupted receiveBlock
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:03:44,823 WARN
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:48654]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:09:42,294 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:37188]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:12:41,892 INFO
> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> >> succeeded for blk_-2674357249542194287_43419
> >> >>
> >> >>
> >> >> Here is our env:
> >> >> hadoop 1.0.3
> >> >> hbase 0.94.1(snappy enabled)
> >> >>
> >> >> java version "1.6.0_31"
> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >> >>
> >> >> # ulimit -a
> >> >> core file size          (blocks, -c) 0
> >> >> data seg size           (kbytes, -d) unlimited
> >> >> scheduling priority             (-e) 20
> >> >> file size               (blocks, -f) unlimited
> >> >> pending signals                 (-i) 16382
> >> >> max locked memory       (kbytes, -l) 64
> >> >> max memory size         (kbytes, -m) unlimited
> >> >> open files                      (-n) 32768
> >> >> pipe size            (512 bytes, -p) 8
> >> >> POSIX message queues     (bytes, -q) 819200
> >> >> real-time priority              (-r) 0
> >> >> stack size              (kbytes, -s) 8192
> >> >> cpu time               (seconds, -t) unlimited
> >> >> max user processes              (-u) 32768
> >> >> virtual memory          (kbytes, -v) unlimited
> >> >> file locks                      (-x) unlimited
> >> >>
> >> >> # uname -a
> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> >> UTC 2011 x86_64 GNU/Linux
> >> >>
> >> >>
> >> >> # free(master)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24732936    8383708   16349228          0     490584
> >> >> 2580356
> >> >> -/+ buffers/cache:    5312768   19420168
> >> >> Swap:     72458232          0   72458232
> >> >>
> >> >>
> >> >> # free(slaves)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24733000   22824276    1908724          0     862556
> >> >> 15303304
> >> >> -/+ buffers/cache:    6658416   18074584
> >> >> Swap:     72458232        264   72457968
> >> >>
> >> >>
> >> >> Some important conf:
> >> >> core-site.xml
> >> >>         <property>
> >> >>                 <name>io.file.buffer.size</name>
> >> >>                 <value>65536</value>
> >> >>         </property>
> >> >>
> >> >> hdfs-site.xml
> >> >>         <property>
> >> >>                 <name>dfs.block.size</name>
> >> >>                 <value>134217728</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.datanode.max.xcievers</name>
> >> >>                 <value>4096</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.support.append</name>
> >> >>                 <value>true</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.replication</name>
> >> >>                 <value>2</value>
> >> >>         </property>
> >> >>
> >> >>
> >> >> Hope you can help us.
> >> >> Thanks in advance.
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Davey Yan
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Network error messages are not always friendly, especially if there is a
misconfiguration.
This said,  "connection refused" says that the network connection was made,
but that the remote port was not opened on the remote box. I.e. the process
was dead.
It could be useful to pastebin the whole logs as well...


On Mon, Feb 25, 2013 at 12:44 PM, Davey Yan <da...@gmail.com> wrote:

> But... there was no log like "network unreachable".
>
>
> On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > I agree.
> > Then for HDFS, ...
> > The first thing to check is the network I would say.
> >
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Thanks for reply, Nicolas.
> >>
> >> My question: What can lead to shutdown of all of the datanodes?
> >> I believe that the regionservers will be OK if the HDFS is OK.
> >>
> >>
> >> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> >> wrote:
> >> > Ok, what's your question?
> >> > When you say the datanode went down, was it the datanode processes or
> >> > the
> >> > machines, with both the datanodes and the regionservers?
> >> >
> >> > The NameNode pings its datanodes every 3 seconds. However it will
> >> > internally
> >> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> >> > have
> >> > 'no answer for x minutes').
> >> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> >> > considered as dead after 180s with no answer. Before, well, it's
> >> > considered
> >> > as live.
> >> > When you stop a regionserver, it tries to flush its data to the disk
> >> > (i.e.
> >> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if
> a
> >> > high
> >> > ratio of your datanodes are dead, it can't shutdown. Connection
> refused
> >> > &
> >> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> >> > does
> >> > not declare the nodes as dead, so hbase tries to use them (and,
> >> > obviously,
> >> > fails). Note that there is now  an intermediate state for hdfs
> >> > datanodes,
> >> > called "stale": an intermediary state where the datanode is used only
> if
> >> > you
> >> > have to (i.e. it's the only datanode with a block replica you need).
> It
> >> > will
> >> > be documented in HBase for the 0.96 release. But if all your datanodes
> >> > are
> >> > down it won't change much.
> >> >
> >> > Cheers,
> >> >
> >> > Nicolas
> >> >
> >> >
> >> >
> >> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com>
> wrote:
> >> >>
> >> >> Hey guys,
> >> >>
> >> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than
> 1
> >> >> year, and it works fine.
> >> >> But the datanodes got shutdown twice in the last month.
> >> >>
> >> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> >> but regionservers of HBase were still live in the HBase web
> >> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> >> All of the processes of jvm were still running, including
> >> >> hmaster/namenode/regionserver/datanode.
> >> >>
> >> >> When the datanodes got shutdown, the load (using the "top" command)
> of
> >> >> slaves became very high, more than 10, higher than normal running.
> >> >> From the "top" command, we saw that the processes of datanode and
> >> >> regionserver were comsuming CPU.
> >> >>
> >> >> We could not stop the HBase or Hadoop cluster through normal
> >> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> >> load of slaves returned to normal level, and we start the cluster
> >> >> again.
> >> >>
> >> >>
> >> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.152:50010
> >> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.149:50010
> >> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.150:50010
> >> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> >> 192.168.1.148:50010
> >> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> >> Removing a node: /default-rack/192.168.1.148:50010
> >> >>
> >> >>
> >> >> Logs in DNs indicated there were many IOException and
> >> >> SocketTimeoutException:
> >> >> 2013-02-22 11:02:52,354 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.io.IOException: Interrupted receiveBlock
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:03:44,823 WARN
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:48654]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:09:42,294 ERROR
> >> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> >> DatanodeRegistration(192.168.1.148:50010,
> >> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> >> for channel to be ready for write. ch :
> >> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> >> remote=/192.168.1.148:37188]
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >> >>         at
> >> >>
> >> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >> >>         at java.lang.Thread.run(Thread.java:662)
> >> >> 2013-02-22 11:12:41,892 INFO
> >> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> >> succeeded for blk_-2674357249542194287_43419
> >> >>
> >> >>
> >> >> Here is our env:
> >> >> hadoop 1.0.3
> >> >> hbase 0.94.1(snappy enabled)
> >> >>
> >> >> java version "1.6.0_31"
> >> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >> >>
> >> >> # ulimit -a
> >> >> core file size          (blocks, -c) 0
> >> >> data seg size           (kbytes, -d) unlimited
> >> >> scheduling priority             (-e) 20
> >> >> file size               (blocks, -f) unlimited
> >> >> pending signals                 (-i) 16382
> >> >> max locked memory       (kbytes, -l) 64
> >> >> max memory size         (kbytes, -m) unlimited
> >> >> open files                      (-n) 32768
> >> >> pipe size            (512 bytes, -p) 8
> >> >> POSIX message queues     (bytes, -q) 819200
> >> >> real-time priority              (-r) 0
> >> >> stack size              (kbytes, -s) 8192
> >> >> cpu time               (seconds, -t) unlimited
> >> >> max user processes              (-u) 32768
> >> >> virtual memory          (kbytes, -v) unlimited
> >> >> file locks                      (-x) unlimited
> >> >>
> >> >> # uname -a
> >> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> >> UTC 2011 x86_64 GNU/Linux
> >> >>
> >> >>
> >> >> # free(master)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24732936    8383708   16349228          0     490584
> >> >> 2580356
> >> >> -/+ buffers/cache:    5312768   19420168
> >> >> Swap:     72458232          0   72458232
> >> >>
> >> >>
> >> >> # free(slaves)
> >> >>              total       used       free     shared    buffers
> >> >> cached
> >> >> Mem:      24733000   22824276    1908724          0     862556
> >> >> 15303304
> >> >> -/+ buffers/cache:    6658416   18074584
> >> >> Swap:     72458232        264   72457968
> >> >>
> >> >>
> >> >> Some important conf:
> >> >> core-site.xml
> >> >>         <property>
> >> >>                 <name>io.file.buffer.size</name>
> >> >>                 <value>65536</value>
> >> >>         </property>
> >> >>
> >> >> hdfs-site.xml
> >> >>         <property>
> >> >>                 <name>dfs.block.size</name>
> >> >>                 <value>134217728</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.datanode.max.xcievers</name>
> >> >>                 <value>4096</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.support.append</name>
> >> >>                 <value>true</value>
> >> >>         </property>
> >> >>         <property>
> >> >>                 <name>dfs.replication</name>
> >> >>                 <value>2</value>
> >> >>         </property>
> >> >>
> >> >>
> >> >> Hope you can help us.
> >> >> Thanks in advance.
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> Davey Yan
> >> >
> >> >
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
But... there was no log like "network unreachable".


On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> I agree.
> Then for HDFS, ...
> The first thing to check is the network I would say.
>
>
>
>
> On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Thanks for reply, Nicolas.
>>
>> My question: What can lead to shutdown of all of the datanodes?
>> I believe that the regionservers will be OK if the HDFS is OK.
>>
>>
>> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > Ok, what's your question?
>> > When you say the datanode went down, was it the datanode processes or
>> > the
>> > machines, with both the datanodes and the regionservers?
>> >
>> > The NameNode pings its datanodes every 3 seconds. However it will
>> > internally
>> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
>> > have
>> > 'no answer for x minutes').
>> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> > considered as dead after 180s with no answer. Before, well, it's
>> > considered
>> > as live.
>> > When you stop a regionserver, it tries to flush its data to the disk
>> > (i.e.
>> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
>> > high
>> > ratio of your datanodes are dead, it can't shutdown. Connection refused
>> > &
>> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> > does
>> > not declare the nodes as dead, so hbase tries to use them (and,
>> > obviously,
>> > fails). Note that there is now  an intermediate state for hdfs
>> > datanodes,
>> > called "stale": an intermediary state where the datanode is used only if
>> > you
>> > have to (i.e. it's the only datanode with a block replica you need). It
>> > will
>> > be documented in HBase for the 0.96 release. But if all your datanodes
>> > are
>> > down it won't change much.
>> >
>> > Cheers,
>> >
>> > Nicolas
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Hey guys,
>> >>
>> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> >> year, and it works fine.
>> >> But the datanodes got shutdown twice in the last month.
>> >>
>> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> but regionservers of HBase were still live in the HBase web
>> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> All of the processes of jvm were still running, including
>> >> hmaster/namenode/regionserver/datanode.
>> >>
>> >> When the datanodes got shutdown, the load (using the "top" command) of
>> >> slaves became very high, more than 10, higher than normal running.
>> >> From the "top" command, we saw that the processes of datanode and
>> >> regionserver were comsuming CPU.
>> >>
>> >> We could not stop the HBase or Hadoop cluster through normal
>> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> load of slaves returned to normal level, and we start the cluster
>> >> again.
>> >>
>> >>
>> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.148:50010
>> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.148:50010
>> >>
>> >>
>> >> Logs in DNs indicated there were many IOException and
>> >> SocketTimeoutException:
>> >> 2013-02-22 11:02:52,354 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.io.IOException: Interrupted receiveBlock
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:03:44,823 WARN
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:48654]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:09:42,294 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:37188]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:12:41,892 INFO
>> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> >> succeeded for blk_-2674357249542194287_43419
>> >>
>> >>
>> >> Here is our env:
>> >> hadoop 1.0.3
>> >> hbase 0.94.1(snappy enabled)
>> >>
>> >> java version "1.6.0_31"
>> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >>
>> >> # ulimit -a
>> >> core file size          (blocks, -c) 0
>> >> data seg size           (kbytes, -d) unlimited
>> >> scheduling priority             (-e) 20
>> >> file size               (blocks, -f) unlimited
>> >> pending signals                 (-i) 16382
>> >> max locked memory       (kbytes, -l) 64
>> >> max memory size         (kbytes, -m) unlimited
>> >> open files                      (-n) 32768
>> >> pipe size            (512 bytes, -p) 8
>> >> POSIX message queues     (bytes, -q) 819200
>> >> real-time priority              (-r) 0
>> >> stack size              (kbytes, -s) 8192
>> >> cpu time               (seconds, -t) unlimited
>> >> max user processes              (-u) 32768
>> >> virtual memory          (kbytes, -v) unlimited
>> >> file locks                      (-x) unlimited
>> >>
>> >> # uname -a
>> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> UTC 2011 x86_64 GNU/Linux
>> >>
>> >>
>> >> # free(master)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24732936    8383708   16349228          0     490584
>> >> 2580356
>> >> -/+ buffers/cache:    5312768   19420168
>> >> Swap:     72458232          0   72458232
>> >>
>> >>
>> >> # free(slaves)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24733000   22824276    1908724          0     862556
>> >> 15303304
>> >> -/+ buffers/cache:    6658416   18074584
>> >> Swap:     72458232        264   72457968
>> >>
>> >>
>> >> Some important conf:
>> >> core-site.xml
>> >>         <property>
>> >>                 <name>io.file.buffer.size</name>
>> >>                 <value>65536</value>
>> >>         </property>
>> >>
>> >> hdfs-site.xml
>> >>         <property>
>> >>                 <name>dfs.block.size</name>
>> >>                 <value>134217728</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.datanode.max.xcievers</name>
>> >>                 <value>4096</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.support.append</name>
>> >>                 <value>true</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.replication</name>
>> >>                 <value>2</value>
>> >>         </property>
>> >>
>> >>
>> >> Hope you can help us.
>> >> Thanks in advance.
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
But... there was no log like "network unreachable".


On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> I agree.
> Then for HDFS, ...
> The first thing to check is the network I would say.
>
>
>
>
> On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Thanks for reply, Nicolas.
>>
>> My question: What can lead to shutdown of all of the datanodes?
>> I believe that the regionservers will be OK if the HDFS is OK.
>>
>>
>> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > Ok, what's your question?
>> > When you say the datanode went down, was it the datanode processes or
>> > the
>> > machines, with both the datanodes and the regionservers?
>> >
>> > The NameNode pings its datanodes every 3 seconds. However it will
>> > internally
>> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
>> > have
>> > 'no answer for x minutes').
>> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> > considered as dead after 180s with no answer. Before, well, it's
>> > considered
>> > as live.
>> > When you stop a regionserver, it tries to flush its data to the disk
>> > (i.e.
>> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
>> > high
>> > ratio of your datanodes are dead, it can't shutdown. Connection refused
>> > &
>> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> > does
>> > not declare the nodes as dead, so hbase tries to use them (and,
>> > obviously,
>> > fails). Note that there is now  an intermediate state for hdfs
>> > datanodes,
>> > called "stale": an intermediary state where the datanode is used only if
>> > you
>> > have to (i.e. it's the only datanode with a block replica you need). It
>> > will
>> > be documented in HBase for the 0.96 release. But if all your datanodes
>> > are
>> > down it won't change much.
>> >
>> > Cheers,
>> >
>> > Nicolas
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Hey guys,
>> >>
>> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> >> year, and it works fine.
>> >> But the datanodes got shutdown twice in the last month.
>> >>
>> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> but regionservers of HBase were still live in the HBase web
>> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> All of the processes of jvm were still running, including
>> >> hmaster/namenode/regionserver/datanode.
>> >>
>> >> When the datanodes got shutdown, the load (using the "top" command) of
>> >> slaves became very high, more than 10, higher than normal running.
>> >> From the "top" command, we saw that the processes of datanode and
>> >> regionserver were comsuming CPU.
>> >>
>> >> We could not stop the HBase or Hadoop cluster through normal
>> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> load of slaves returned to normal level, and we start the cluster
>> >> again.
>> >>
>> >>
>> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.148:50010
>> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.148:50010
>> >>
>> >>
>> >> Logs in DNs indicated there were many IOException and
>> >> SocketTimeoutException:
>> >> 2013-02-22 11:02:52,354 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.io.IOException: Interrupted receiveBlock
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:03:44,823 WARN
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:48654]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:09:42,294 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:37188]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:12:41,892 INFO
>> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> >> succeeded for blk_-2674357249542194287_43419
>> >>
>> >>
>> >> Here is our env:
>> >> hadoop 1.0.3
>> >> hbase 0.94.1(snappy enabled)
>> >>
>> >> java version "1.6.0_31"
>> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >>
>> >> # ulimit -a
>> >> core file size          (blocks, -c) 0
>> >> data seg size           (kbytes, -d) unlimited
>> >> scheduling priority             (-e) 20
>> >> file size               (blocks, -f) unlimited
>> >> pending signals                 (-i) 16382
>> >> max locked memory       (kbytes, -l) 64
>> >> max memory size         (kbytes, -m) unlimited
>> >> open files                      (-n) 32768
>> >> pipe size            (512 bytes, -p) 8
>> >> POSIX message queues     (bytes, -q) 819200
>> >> real-time priority              (-r) 0
>> >> stack size              (kbytes, -s) 8192
>> >> cpu time               (seconds, -t) unlimited
>> >> max user processes              (-u) 32768
>> >> virtual memory          (kbytes, -v) unlimited
>> >> file locks                      (-x) unlimited
>> >>
>> >> # uname -a
>> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> UTC 2011 x86_64 GNU/Linux
>> >>
>> >>
>> >> # free(master)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24732936    8383708   16349228          0     490584
>> >> 2580356
>> >> -/+ buffers/cache:    5312768   19420168
>> >> Swap:     72458232          0   72458232
>> >>
>> >>
>> >> # free(slaves)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24733000   22824276    1908724          0     862556
>> >> 15303304
>> >> -/+ buffers/cache:    6658416   18074584
>> >> Swap:     72458232        264   72457968
>> >>
>> >>
>> >> Some important conf:
>> >> core-site.xml
>> >>         <property>
>> >>                 <name>io.file.buffer.size</name>
>> >>                 <value>65536</value>
>> >>         </property>
>> >>
>> >> hdfs-site.xml
>> >>         <property>
>> >>                 <name>dfs.block.size</name>
>> >>                 <value>134217728</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.datanode.max.xcievers</name>
>> >>                 <value>4096</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.support.append</name>
>> >>                 <value>true</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.replication</name>
>> >>                 <value>2</value>
>> >>         </property>
>> >>
>> >>
>> >> Hope you can help us.
>> >> Thanks in advance.
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
But... there was no log like "network unreachable".


On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> I agree.
> Then for HDFS, ...
> The first thing to check is the network I would say.
>
>
>
>
> On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Thanks for reply, Nicolas.
>>
>> My question: What can lead to shutdown of all of the datanodes?
>> I believe that the regionservers will be OK if the HDFS is OK.
>>
>>
>> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > Ok, what's your question?
>> > When you say the datanode went down, was it the datanode processes or
>> > the
>> > machines, with both the datanodes and the regionservers?
>> >
>> > The NameNode pings its datanodes every 3 seconds. However it will
>> > internally
>> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
>> > have
>> > 'no answer for x minutes').
>> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> > considered as dead after 180s with no answer. Before, well, it's
>> > considered
>> > as live.
>> > When you stop a regionserver, it tries to flush its data to the disk
>> > (i.e.
>> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
>> > high
>> > ratio of your datanodes are dead, it can't shutdown. Connection refused
>> > &
>> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> > does
>> > not declare the nodes as dead, so hbase tries to use them (and,
>> > obviously,
>> > fails). Note that there is now  an intermediate state for hdfs
>> > datanodes,
>> > called "stale": an intermediary state where the datanode is used only if
>> > you
>> > have to (i.e. it's the only datanode with a block replica you need). It
>> > will
>> > be documented in HBase for the 0.96 release. But if all your datanodes
>> > are
>> > down it won't change much.
>> >
>> > Cheers,
>> >
>> > Nicolas
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Hey guys,
>> >>
>> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> >> year, and it works fine.
>> >> But the datanodes got shutdown twice in the last month.
>> >>
>> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> but regionservers of HBase were still live in the HBase web
>> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> All of the processes of jvm were still running, including
>> >> hmaster/namenode/regionserver/datanode.
>> >>
>> >> When the datanodes got shutdown, the load (using the "top" command) of
>> >> slaves became very high, more than 10, higher than normal running.
>> >> From the "top" command, we saw that the processes of datanode and
>> >> regionserver were comsuming CPU.
>> >>
>> >> We could not stop the HBase or Hadoop cluster through normal
>> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> load of slaves returned to normal level, and we start the cluster
>> >> again.
>> >>
>> >>
>> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.148:50010
>> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.148:50010
>> >>
>> >>
>> >> Logs in DNs indicated there were many IOException and
>> >> SocketTimeoutException:
>> >> 2013-02-22 11:02:52,354 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.io.IOException: Interrupted receiveBlock
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:03:44,823 WARN
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:48654]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:09:42,294 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:37188]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:12:41,892 INFO
>> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> >> succeeded for blk_-2674357249542194287_43419
>> >>
>> >>
>> >> Here is our env:
>> >> hadoop 1.0.3
>> >> hbase 0.94.1(snappy enabled)
>> >>
>> >> java version "1.6.0_31"
>> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >>
>> >> # ulimit -a
>> >> core file size          (blocks, -c) 0
>> >> data seg size           (kbytes, -d) unlimited
>> >> scheduling priority             (-e) 20
>> >> file size               (blocks, -f) unlimited
>> >> pending signals                 (-i) 16382
>> >> max locked memory       (kbytes, -l) 64
>> >> max memory size         (kbytes, -m) unlimited
>> >> open files                      (-n) 32768
>> >> pipe size            (512 bytes, -p) 8
>> >> POSIX message queues     (bytes, -q) 819200
>> >> real-time priority              (-r) 0
>> >> stack size              (kbytes, -s) 8192
>> >> cpu time               (seconds, -t) unlimited
>> >> max user processes              (-u) 32768
>> >> virtual memory          (kbytes, -v) unlimited
>> >> file locks                      (-x) unlimited
>> >>
>> >> # uname -a
>> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> UTC 2011 x86_64 GNU/Linux
>> >>
>> >>
>> >> # free(master)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24732936    8383708   16349228          0     490584
>> >> 2580356
>> >> -/+ buffers/cache:    5312768   19420168
>> >> Swap:     72458232          0   72458232
>> >>
>> >>
>> >> # free(slaves)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24733000   22824276    1908724          0     862556
>> >> 15303304
>> >> -/+ buffers/cache:    6658416   18074584
>> >> Swap:     72458232        264   72457968
>> >>
>> >>
>> >> Some important conf:
>> >> core-site.xml
>> >>         <property>
>> >>                 <name>io.file.buffer.size</name>
>> >>                 <value>65536</value>
>> >>         </property>
>> >>
>> >> hdfs-site.xml
>> >>         <property>
>> >>                 <name>dfs.block.size</name>
>> >>                 <value>134217728</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.datanode.max.xcievers</name>
>> >>                 <value>4096</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.support.append</name>
>> >>                 <value>true</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.replication</name>
>> >>                 <value>2</value>
>> >>         </property>
>> >>
>> >>
>> >> Hope you can help us.
>> >> Thanks in advance.
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
But... there was no log like "network unreachable".


On Mon, Feb 25, 2013 at 6:07 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> I agree.
> Then for HDFS, ...
> The first thing to check is the network I would say.
>
>
>
>
> On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Thanks for reply, Nicolas.
>>
>> My question: What can lead to shutdown of all of the datanodes?
>> I believe that the regionservers will be OK if the HDFS is OK.
>>
>>
>> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
>> wrote:
>> > Ok, what's your question?
>> > When you say the datanode went down, was it the datanode processes or
>> > the
>> > machines, with both the datanodes and the regionservers?
>> >
>> > The NameNode pings its datanodes every 3 seconds. However it will
>> > internally
>> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
>> > have
>> > 'no answer for x minutes').
>> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
>> > considered as dead after 180s with no answer. Before, well, it's
>> > considered
>> > as live.
>> > When you stop a regionserver, it tries to flush its data to the disk
>> > (i.e.
>> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
>> > high
>> > ratio of your datanodes are dead, it can't shutdown. Connection refused
>> > &
>> > socket timeouts come from the fact that before the 10:30 minutes hdfs
>> > does
>> > not declare the nodes as dead, so hbase tries to use them (and,
>> > obviously,
>> > fails). Note that there is now  an intermediate state for hdfs
>> > datanodes,
>> > called "stale": an intermediary state where the datanode is used only if
>> > you
>> > have to (i.e. it's the only datanode with a block replica you need). It
>> > will
>> > be documented in HBase for the 0.96 release. But if all your datanodes
>> > are
>> > down it won't change much.
>> >
>> > Cheers,
>> >
>> > Nicolas
>> >
>> >
>> >
>> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>> >>
>> >> Hey guys,
>> >>
>> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> >> year, and it works fine.
>> >> But the datanodes got shutdown twice in the last month.
>> >>
>> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> >> but regionservers of HBase were still live in the HBase web
>> >> admin(http://ip:60010/master-status), of course, they were zombies.
>> >> All of the processes of jvm were still running, including
>> >> hmaster/namenode/regionserver/datanode.
>> >>
>> >> When the datanodes got shutdown, the load (using the "top" command) of
>> >> slaves became very high, more than 10, higher than normal running.
>> >> From the "top" command, we saw that the processes of datanode and
>> >> regionserver were comsuming CPU.
>> >>
>> >> We could not stop the HBase or Hadoop cluster through normal
>> >> commands(stop-*.sh/*-daemon.sh stop *).
>> >> So we stopped datanodes and regionservers by kill -9 PID, then the
>> >> load of slaves returned to normal level, and we start the cluster
>> >> again.
>> >>
>> >>
>> >> Log of NN at the shutdown point(All of the DNs were removed):
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.152:50010
>> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.149:50010
>> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.150:50010
>> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> >> 192.168.1.148:50010
>> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> >> Removing a node: /default-rack/192.168.1.148:50010
>> >>
>> >>
>> >> Logs in DNs indicated there were many IOException and
>> >> SocketTimeoutException:
>> >> 2013-02-22 11:02:52,354 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.io.IOException: Interrupted receiveBlock
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:03:44,823 WARN
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):Got exception while serving
>> >> blk_-1985405101514576650_247001 to /192.168.1.148:
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:48654]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:09:42,294 ERROR
>> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> >> DatanodeRegistration(192.168.1.148:50010,
>> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> >> infoPort=50075, ipcPort=50020):DataXceiver
>> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> >> for channel to be ready for write. ch :
>> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> >> remote=/192.168.1.148:37188]
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>> >>         at
>> >>
>> >> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>> >>         at
>> >>
>> >> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>> >>         at java.lang.Thread.run(Thread.java:662)
>> >> 2013-02-22 11:12:41,892 INFO
>> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> >> succeeded for blk_-2674357249542194287_43419
>> >>
>> >>
>> >> Here is our env:
>> >> hadoop 1.0.3
>> >> hbase 0.94.1(snappy enabled)
>> >>
>> >> java version "1.6.0_31"
>> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>> >>
>> >> # ulimit -a
>> >> core file size          (blocks, -c) 0
>> >> data seg size           (kbytes, -d) unlimited
>> >> scheduling priority             (-e) 20
>> >> file size               (blocks, -f) unlimited
>> >> pending signals                 (-i) 16382
>> >> max locked memory       (kbytes, -l) 64
>> >> max memory size         (kbytes, -m) unlimited
>> >> open files                      (-n) 32768
>> >> pipe size            (512 bytes, -p) 8
>> >> POSIX message queues     (bytes, -q) 819200
>> >> real-time priority              (-r) 0
>> >> stack size              (kbytes, -s) 8192
>> >> cpu time               (seconds, -t) unlimited
>> >> max user processes              (-u) 32768
>> >> virtual memory          (kbytes, -v) unlimited
>> >> file locks                      (-x) unlimited
>> >>
>> >> # uname -a
>> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> >> UTC 2011 x86_64 GNU/Linux
>> >>
>> >>
>> >> # free(master)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24732936    8383708   16349228          0     490584
>> >> 2580356
>> >> -/+ buffers/cache:    5312768   19420168
>> >> Swap:     72458232          0   72458232
>> >>
>> >>
>> >> # free(slaves)
>> >>              total       used       free     shared    buffers
>> >> cached
>> >> Mem:      24733000   22824276    1908724          0     862556
>> >> 15303304
>> >> -/+ buffers/cache:    6658416   18074584
>> >> Swap:     72458232        264   72457968
>> >>
>> >>
>> >> Some important conf:
>> >> core-site.xml
>> >>         <property>
>> >>                 <name>io.file.buffer.size</name>
>> >>                 <value>65536</value>
>> >>         </property>
>> >>
>> >> hdfs-site.xml
>> >>         <property>
>> >>                 <name>dfs.block.size</name>
>> >>                 <value>134217728</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.datanode.max.xcievers</name>
>> >>                 <value>4096</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.support.append</name>
>> >>                 <value>true</value>
>> >>         </property>
>> >>         <property>
>> >>                 <name>dfs.replication</name>
>> >>                 <value>2</value>
>> >>         </property>
>> >>
>> >>
>> >> Hope you can help us.
>> >> Thanks in advance.
>> >>
>> >>
>> >>
>> >> --
>> >> Davey Yan
>> >
>> >
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
I agree.
Then for HDFS, ...
The first thing to check is the network I would say.




On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:

> Thanks for reply, Nicolas.
>
> My question: What can lead to shutdown of all of the datanodes?
> I believe that the regionservers will be OK if the HDFS is OK.
>
>
> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > Ok, what's your question?
> > When you say the datanode went down, was it the datanode processes or the
> > machines, with both the datanodes and the regionservers?
> >
> > The NameNode pings its datanodes every 3 seconds. However it will
> internally
> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> have
> > 'no answer for x minutes').
> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> > considered as dead after 180s with no answer. Before, well, it's
> considered
> > as live.
> > When you stop a regionserver, it tries to flush its data to the disk
> (i.e.
> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
> high
> > ratio of your datanodes are dead, it can't shutdown. Connection refused &
> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> does
> > not declare the nodes as dead, so hbase tries to use them (and,
> obviously,
> > fails). Note that there is now  an intermediate state for hdfs datanodes,
> > called "stale": an intermediary state where the datanode is used only if
> you
> > have to (i.e. it's the only datanode with a block replica you need). It
> will
> > be documented in HBase for the 0.96 release. But if all your datanodes
> are
> > down it won't change much.
> >
> > Cheers,
> >
> > Nicolas
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Hey guys,
> >>
> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> >> year, and it works fine.
> >> But the datanodes got shutdown twice in the last month.
> >>
> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> but regionservers of HBase were still live in the HBase web
> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> All of the processes of jvm were still running, including
> >> hmaster/namenode/regionserver/datanode.
> >>
> >> When the datanodes got shutdown, the load (using the "top" command) of
> >> slaves became very high, more than 10, higher than normal running.
> >> From the "top" command, we saw that the processes of datanode and
> >> regionserver were comsuming CPU.
> >>
> >> We could not stop the HBase or Hadoop cluster through normal
> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> load of slaves returned to normal level, and we start the cluster
> >> again.
> >>
> >>
> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.152:50010
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.148:50010
> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.148:50010
> >>
> >>
> >> Logs in DNs indicated there were many IOException and
> >> SocketTimeoutException:
> >> 2013-02-22 11:02:52,354 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.io.IOException: Interrupted receiveBlock
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:03:44,823 WARN
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:48654]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:09:42,294 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:37188]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:12:41,892 INFO
> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> succeeded for blk_-2674357249542194287_43419
> >>
> >>
> >> Here is our env:
> >> hadoop 1.0.3
> >> hbase 0.94.1(snappy enabled)
> >>
> >> java version "1.6.0_31"
> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >>
> >> # ulimit -a
> >> core file size          (blocks, -c) 0
> >> data seg size           (kbytes, -d) unlimited
> >> scheduling priority             (-e) 20
> >> file size               (blocks, -f) unlimited
> >> pending signals                 (-i) 16382
> >> max locked memory       (kbytes, -l) 64
> >> max memory size         (kbytes, -m) unlimited
> >> open files                      (-n) 32768
> >> pipe size            (512 bytes, -p) 8
> >> POSIX message queues     (bytes, -q) 819200
> >> real-time priority              (-r) 0
> >> stack size              (kbytes, -s) 8192
> >> cpu time               (seconds, -t) unlimited
> >> max user processes              (-u) 32768
> >> virtual memory          (kbytes, -v) unlimited
> >> file locks                      (-x) unlimited
> >>
> >> # uname -a
> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> UTC 2011 x86_64 GNU/Linux
> >>
> >>
> >> # free(master)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24732936    8383708   16349228          0     490584
>  2580356
> >> -/+ buffers/cache:    5312768   19420168
> >> Swap:     72458232          0   72458232
> >>
> >>
> >> # free(slaves)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24733000   22824276    1908724          0     862556
> 15303304
> >> -/+ buffers/cache:    6658416   18074584
> >> Swap:     72458232        264   72457968
> >>
> >>
> >> Some important conf:
> >> core-site.xml
> >>         <property>
> >>                 <name>io.file.buffer.size</name>
> >>                 <value>65536</value>
> >>         </property>
> >>
> >> hdfs-site.xml
> >>         <property>
> >>                 <name>dfs.block.size</name>
> >>                 <value>134217728</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.datanode.max.xcievers</name>
> >>                 <value>4096</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.support.append</name>
> >>                 <value>true</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.replication</name>
> >>                 <value>2</value>
> >>         </property>
> >>
> >>
> >> Hope you can help us.
> >> Thanks in advance.
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
I agree.
Then for HDFS, ...
The first thing to check is the network I would say.




On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:

> Thanks for reply, Nicolas.
>
> My question: What can lead to shutdown of all of the datanodes?
> I believe that the regionservers will be OK if the HDFS is OK.
>
>
> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > Ok, what's your question?
> > When you say the datanode went down, was it the datanode processes or the
> > machines, with both the datanodes and the regionservers?
> >
> > The NameNode pings its datanodes every 3 seconds. However it will
> internally
> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> have
> > 'no answer for x minutes').
> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> > considered as dead after 180s with no answer. Before, well, it's
> considered
> > as live.
> > When you stop a regionserver, it tries to flush its data to the disk
> (i.e.
> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
> high
> > ratio of your datanodes are dead, it can't shutdown. Connection refused &
> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> does
> > not declare the nodes as dead, so hbase tries to use them (and,
> obviously,
> > fails). Note that there is now  an intermediate state for hdfs datanodes,
> > called "stale": an intermediary state where the datanode is used only if
> you
> > have to (i.e. it's the only datanode with a block replica you need). It
> will
> > be documented in HBase for the 0.96 release. But if all your datanodes
> are
> > down it won't change much.
> >
> > Cheers,
> >
> > Nicolas
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Hey guys,
> >>
> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> >> year, and it works fine.
> >> But the datanodes got shutdown twice in the last month.
> >>
> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> but regionservers of HBase were still live in the HBase web
> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> All of the processes of jvm were still running, including
> >> hmaster/namenode/regionserver/datanode.
> >>
> >> When the datanodes got shutdown, the load (using the "top" command) of
> >> slaves became very high, more than 10, higher than normal running.
> >> From the "top" command, we saw that the processes of datanode and
> >> regionserver were comsuming CPU.
> >>
> >> We could not stop the HBase or Hadoop cluster through normal
> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> load of slaves returned to normal level, and we start the cluster
> >> again.
> >>
> >>
> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.152:50010
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.148:50010
> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.148:50010
> >>
> >>
> >> Logs in DNs indicated there were many IOException and
> >> SocketTimeoutException:
> >> 2013-02-22 11:02:52,354 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.io.IOException: Interrupted receiveBlock
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:03:44,823 WARN
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:48654]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:09:42,294 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:37188]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:12:41,892 INFO
> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> succeeded for blk_-2674357249542194287_43419
> >>
> >>
> >> Here is our env:
> >> hadoop 1.0.3
> >> hbase 0.94.1(snappy enabled)
> >>
> >> java version "1.6.0_31"
> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >>
> >> # ulimit -a
> >> core file size          (blocks, -c) 0
> >> data seg size           (kbytes, -d) unlimited
> >> scheduling priority             (-e) 20
> >> file size               (blocks, -f) unlimited
> >> pending signals                 (-i) 16382
> >> max locked memory       (kbytes, -l) 64
> >> max memory size         (kbytes, -m) unlimited
> >> open files                      (-n) 32768
> >> pipe size            (512 bytes, -p) 8
> >> POSIX message queues     (bytes, -q) 819200
> >> real-time priority              (-r) 0
> >> stack size              (kbytes, -s) 8192
> >> cpu time               (seconds, -t) unlimited
> >> max user processes              (-u) 32768
> >> virtual memory          (kbytes, -v) unlimited
> >> file locks                      (-x) unlimited
> >>
> >> # uname -a
> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> UTC 2011 x86_64 GNU/Linux
> >>
> >>
> >> # free(master)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24732936    8383708   16349228          0     490584
>  2580356
> >> -/+ buffers/cache:    5312768   19420168
> >> Swap:     72458232          0   72458232
> >>
> >>
> >> # free(slaves)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24733000   22824276    1908724          0     862556
> 15303304
> >> -/+ buffers/cache:    6658416   18074584
> >> Swap:     72458232        264   72457968
> >>
> >>
> >> Some important conf:
> >> core-site.xml
> >>         <property>
> >>                 <name>io.file.buffer.size</name>
> >>                 <value>65536</value>
> >>         </property>
> >>
> >> hdfs-site.xml
> >>         <property>
> >>                 <name>dfs.block.size</name>
> >>                 <value>134217728</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.datanode.max.xcievers</name>
> >>                 <value>4096</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.support.append</name>
> >>                 <value>true</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.replication</name>
> >>                 <value>2</value>
> >>         </property>
> >>
> >>
> >> Hope you can help us.
> >> Thanks in advance.
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
I agree.
Then for HDFS, ...
The first thing to check is the network I would say.




On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:

> Thanks for reply, Nicolas.
>
> My question: What can lead to shutdown of all of the datanodes?
> I believe that the regionservers will be OK if the HDFS is OK.
>
>
> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > Ok, what's your question?
> > When you say the datanode went down, was it the datanode processes or the
> > machines, with both the datanodes and the regionservers?
> >
> > The NameNode pings its datanodes every 3 seconds. However it will
> internally
> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> have
> > 'no answer for x minutes').
> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> > considered as dead after 180s with no answer. Before, well, it's
> considered
> > as live.
> > When you stop a regionserver, it tries to flush its data to the disk
> (i.e.
> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
> high
> > ratio of your datanodes are dead, it can't shutdown. Connection refused &
> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> does
> > not declare the nodes as dead, so hbase tries to use them (and,
> obviously,
> > fails). Note that there is now  an intermediate state for hdfs datanodes,
> > called "stale": an intermediary state where the datanode is used only if
> you
> > have to (i.e. it's the only datanode with a block replica you need). It
> will
> > be documented in HBase for the 0.96 release. But if all your datanodes
> are
> > down it won't change much.
> >
> > Cheers,
> >
> > Nicolas
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Hey guys,
> >>
> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> >> year, and it works fine.
> >> But the datanodes got shutdown twice in the last month.
> >>
> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> but regionservers of HBase were still live in the HBase web
> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> All of the processes of jvm were still running, including
> >> hmaster/namenode/regionserver/datanode.
> >>
> >> When the datanodes got shutdown, the load (using the "top" command) of
> >> slaves became very high, more than 10, higher than normal running.
> >> From the "top" command, we saw that the processes of datanode and
> >> regionserver were comsuming CPU.
> >>
> >> We could not stop the HBase or Hadoop cluster through normal
> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> load of slaves returned to normal level, and we start the cluster
> >> again.
> >>
> >>
> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.152:50010
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.148:50010
> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.148:50010
> >>
> >>
> >> Logs in DNs indicated there were many IOException and
> >> SocketTimeoutException:
> >> 2013-02-22 11:02:52,354 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.io.IOException: Interrupted receiveBlock
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:03:44,823 WARN
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:48654]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:09:42,294 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:37188]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:12:41,892 INFO
> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> succeeded for blk_-2674357249542194287_43419
> >>
> >>
> >> Here is our env:
> >> hadoop 1.0.3
> >> hbase 0.94.1(snappy enabled)
> >>
> >> java version "1.6.0_31"
> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >>
> >> # ulimit -a
> >> core file size          (blocks, -c) 0
> >> data seg size           (kbytes, -d) unlimited
> >> scheduling priority             (-e) 20
> >> file size               (blocks, -f) unlimited
> >> pending signals                 (-i) 16382
> >> max locked memory       (kbytes, -l) 64
> >> max memory size         (kbytes, -m) unlimited
> >> open files                      (-n) 32768
> >> pipe size            (512 bytes, -p) 8
> >> POSIX message queues     (bytes, -q) 819200
> >> real-time priority              (-r) 0
> >> stack size              (kbytes, -s) 8192
> >> cpu time               (seconds, -t) unlimited
> >> max user processes              (-u) 32768
> >> virtual memory          (kbytes, -v) unlimited
> >> file locks                      (-x) unlimited
> >>
> >> # uname -a
> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> UTC 2011 x86_64 GNU/Linux
> >>
> >>
> >> # free(master)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24732936    8383708   16349228          0     490584
>  2580356
> >> -/+ buffers/cache:    5312768   19420168
> >> Swap:     72458232          0   72458232
> >>
> >>
> >> # free(slaves)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24733000   22824276    1908724          0     862556
> 15303304
> >> -/+ buffers/cache:    6658416   18074584
> >> Swap:     72458232        264   72457968
> >>
> >>
> >> Some important conf:
> >> core-site.xml
> >>         <property>
> >>                 <name>io.file.buffer.size</name>
> >>                 <value>65536</value>
> >>         </property>
> >>
> >> hdfs-site.xml
> >>         <property>
> >>                 <name>dfs.block.size</name>
> >>                 <value>134217728</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.datanode.max.xcievers</name>
> >>                 <value>4096</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.support.append</name>
> >>                 <value>true</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.replication</name>
> >>                 <value>2</value>
> >>         </property>
> >>
> >>
> >> Hope you can help us.
> >> Thanks in advance.
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
I agree.
Then for HDFS, ...
The first thing to check is the network I would say.




On Mon, Feb 25, 2013 at 10:46 AM, Davey Yan <da...@gmail.com> wrote:

> Thanks for reply, Nicolas.
>
> My question: What can lead to shutdown of all of the datanodes?
> I believe that the regionservers will be OK if the HDFS is OK.
>
>
> On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com>
> wrote:
> > Ok, what's your question?
> > When you say the datanode went down, was it the datanode processes or the
> > machines, with both the datanodes and the regionservers?
> >
> > The NameNode pings its datanodes every 3 seconds. However it will
> internally
> > mark the datanodes as dead after 10:30 minutes (even if in the gui you
> have
> > 'no answer for x minutes').
> > HBase monitoring is done by ZooKeeper. By default, a regionserver is
> > considered as dead after 180s with no answer. Before, well, it's
> considered
> > as live.
> > When you stop a regionserver, it tries to flush its data to the disk
> (i.e.
> > hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
> high
> > ratio of your datanodes are dead, it can't shutdown. Connection refused &
> > socket timeouts come from the fact that before the 10:30 minutes hdfs
> does
> > not declare the nodes as dead, so hbase tries to use them (and,
> obviously,
> > fails). Note that there is now  an intermediate state for hdfs datanodes,
> > called "stale": an intermediary state where the datanode is used only if
> you
> > have to (i.e. it's the only datanode with a block replica you need). It
> will
> > be documented in HBase for the 0.96 release. But if all your datanodes
> are
> > down it won't change much.
> >
> > Cheers,
> >
> > Nicolas
> >
> >
> >
> > On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
> >>
> >> Hey guys,
> >>
> >> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> >> year, and it works fine.
> >> But the datanodes got shutdown twice in the last month.
> >>
> >> When the datanodes got shutdown, all of them became "Dead Nodes" in
> >> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> >> but regionservers of HBase were still live in the HBase web
> >> admin(http://ip:60010/master-status), of course, they were zombies.
> >> All of the processes of jvm were still running, including
> >> hmaster/namenode/regionserver/datanode.
> >>
> >> When the datanodes got shutdown, the load (using the "top" command) of
> >> slaves became very high, more than 10, higher than normal running.
> >> From the "top" command, we saw that the processes of datanode and
> >> regionserver were comsuming CPU.
> >>
> >> We could not stop the HBase or Hadoop cluster through normal
> >> commands(stop-*.sh/*-daemon.sh stop *).
> >> So we stopped datanodes and regionservers by kill -9 PID, then the
> >> load of slaves returned to normal level, and we start the cluster
> >> again.
> >>
> >>
> >> Log of NN at the shutdown point(All of the DNs were removed):
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.152:50010
> >> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.149:50010
> >> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.150:50010
> >> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> >> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> >> 192.168.1.148:50010
> >> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> >> Removing a node: /default-rack/192.168.1.148:50010
> >>
> >>
> >> Logs in DNs indicated there were many IOException and
> >> SocketTimeoutException:
> >> 2013-02-22 11:02:52,354 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.io.IOException: Interrupted receiveBlock
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:03:44,823 WARN
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):Got exception while serving
> >> blk_-1985405101514576650_247001 to /192.168.1.148:
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:48654]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:09:42,294 ERROR
> >> org.apache.hadoop.hdfs.server.datanode.DataNode:
> >> DatanodeRegistration(192.168.1.148:50010,
> >> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> >> infoPort=50075, ipcPort=50020):DataXceiver
> >> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> >> for channel to be ready for write. ch :
> >> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> >> remote=/192.168.1.148:37188]
> >>         at
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
> >>         at
> >>
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
> >>         at
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
> >>         at java.lang.Thread.run(Thread.java:662)
> >> 2013-02-22 11:12:41,892 INFO
> >> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> >> succeeded for blk_-2674357249542194287_43419
> >>
> >>
> >> Here is our env:
> >> hadoop 1.0.3
> >> hbase 0.94.1(snappy enabled)
> >>
> >> java version "1.6.0_31"
> >> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> >> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
> >>
> >> # ulimit -a
> >> core file size          (blocks, -c) 0
> >> data seg size           (kbytes, -d) unlimited
> >> scheduling priority             (-e) 20
> >> file size               (blocks, -f) unlimited
> >> pending signals                 (-i) 16382
> >> max locked memory       (kbytes, -l) 64
> >> max memory size         (kbytes, -m) unlimited
> >> open files                      (-n) 32768
> >> pipe size            (512 bytes, -p) 8
> >> POSIX message queues     (bytes, -q) 819200
> >> real-time priority              (-r) 0
> >> stack size              (kbytes, -s) 8192
> >> cpu time               (seconds, -t) unlimited
> >> max user processes              (-u) 32768
> >> virtual memory          (kbytes, -v) unlimited
> >> file locks                      (-x) unlimited
> >>
> >> # uname -a
> >> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> >> UTC 2011 x86_64 GNU/Linux
> >>
> >>
> >> # free(master)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24732936    8383708   16349228          0     490584
>  2580356
> >> -/+ buffers/cache:    5312768   19420168
> >> Swap:     72458232          0   72458232
> >>
> >>
> >> # free(slaves)
> >>              total       used       free     shared    buffers
> cached
> >> Mem:      24733000   22824276    1908724          0     862556
> 15303304
> >> -/+ buffers/cache:    6658416   18074584
> >> Swap:     72458232        264   72457968
> >>
> >>
> >> Some important conf:
> >> core-site.xml
> >>         <property>
> >>                 <name>io.file.buffer.size</name>
> >>                 <value>65536</value>
> >>         </property>
> >>
> >> hdfs-site.xml
> >>         <property>
> >>                 <name>dfs.block.size</name>
> >>                 <value>134217728</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.datanode.max.xcievers</name>
> >>                 <value>4096</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.support.append</name>
> >>                 <value>true</value>
> >>         </property>
> >>         <property>
> >>                 <name>dfs.replication</name>
> >>                 <value>2</value>
> >>         </property>
> >>
> >>
> >> Hope you can help us.
> >> Thanks in advance.
> >>
> >>
> >>
> >> --
> >> Davey Yan
> >
> >
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Thanks for reply, Nicolas.

My question: What can lead to shutdown of all of the datanodes?
I believe that the regionservers will be OK if the HDFS is OK.


On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Ok, what's your question?
> When you say the datanode went down, was it the datanode processes or the
> machines, with both the datanodes and the regionservers?
>
> The NameNode pings its datanodes every 3 seconds. However it will internally
> mark the datanodes as dead after 10:30 minutes (even if in the gui you have
> 'no answer for x minutes').
> HBase monitoring is done by ZooKeeper. By default, a regionserver is
> considered as dead after 180s with no answer. Before, well, it's considered
> as live.
> When you stop a regionserver, it tries to flush its data to the disk (i.e.
> hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a high
> ratio of your datanodes are dead, it can't shutdown. Connection refused &
> socket timeouts come from the fact that before the 10:30 minutes hdfs does
> not declare the nodes as dead, so hbase tries to use them (and, obviously,
> fails). Note that there is now  an intermediate state for hdfs datanodes,
> called "stale": an intermediary state where the datanode is used only if you
> have to (i.e. it's the only datanode with a block replica you need). It will
> be documented in HBase for the 0.96 release. But if all your datanodes are
> down it won't change much.
>
> Cheers,
>
> Nicolas
>
>
>
> On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Hey guys,
>>
>> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> year, and it works fine.
>> But the datanodes got shutdown twice in the last month.
>>
>> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> but regionservers of HBase were still live in the HBase web
>> admin(http://ip:60010/master-status), of course, they were zombies.
>> All of the processes of jvm were still running, including
>> hmaster/namenode/regionserver/datanode.
>>
>> When the datanodes got shutdown, the load (using the "top" command) of
>> slaves became very high, more than 10, higher than normal running.
>> From the "top" command, we saw that the processes of datanode and
>> regionserver were comsuming CPU.
>>
>> We could not stop the HBase or Hadoop cluster through normal
>> commands(stop-*.sh/*-daemon.sh stop *).
>> So we stopped datanodes and regionservers by kill -9 PID, then the
>> load of slaves returned to normal level, and we start the cluster
>> again.
>>
>>
>> Log of NN at the shutdown point(All of the DNs were removed):
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.152:50010
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.148:50010
>> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.148:50010
>>
>>
>> Logs in DNs indicated there were many IOException and
>> SocketTimeoutException:
>> 2013-02-22 11:02:52,354 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.io.IOException: Interrupted receiveBlock
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:03:44,823 WARN
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):Got exception while serving
>> blk_-1985405101514576650_247001 to /192.168.1.148:
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:48654]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:09:42,294 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:37188]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:12:41,892 INFO
>> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> succeeded for blk_-2674357249542194287_43419
>>
>>
>> Here is our env:
>> hadoop 1.0.3
>> hbase 0.94.1(snappy enabled)
>>
>> java version "1.6.0_31"
>> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>
>> # ulimit -a
>> core file size          (blocks, -c) 0
>> data seg size           (kbytes, -d) unlimited
>> scheduling priority             (-e) 20
>> file size               (blocks, -f) unlimited
>> pending signals                 (-i) 16382
>> max locked memory       (kbytes, -l) 64
>> max memory size         (kbytes, -m) unlimited
>> open files                      (-n) 32768
>> pipe size            (512 bytes, -p) 8
>> POSIX message queues     (bytes, -q) 819200
>> real-time priority              (-r) 0
>> stack size              (kbytes, -s) 8192
>> cpu time               (seconds, -t) unlimited
>> max user processes              (-u) 32768
>> virtual memory          (kbytes, -v) unlimited
>> file locks                      (-x) unlimited
>>
>> # uname -a
>> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> UTC 2011 x86_64 GNU/Linux
>>
>>
>> # free(master)
>>              total       used       free     shared    buffers     cached
>> Mem:      24732936    8383708   16349228          0     490584    2580356
>> -/+ buffers/cache:    5312768   19420168
>> Swap:     72458232          0   72458232
>>
>>
>> # free(slaves)
>>              total       used       free     shared    buffers     cached
>> Mem:      24733000   22824276    1908724          0     862556   15303304
>> -/+ buffers/cache:    6658416   18074584
>> Swap:     72458232        264   72457968
>>
>>
>> Some important conf:
>> core-site.xml
>>         <property>
>>                 <name>io.file.buffer.size</name>
>>                 <value>65536</value>
>>         </property>
>>
>> hdfs-site.xml
>>         <property>
>>                 <name>dfs.block.size</name>
>>                 <value>134217728</value>
>>         </property>
>>         <property>
>>                 <name>dfs.datanode.max.xcievers</name>
>>                 <value>4096</value>
>>         </property>
>>         <property>
>>                 <name>dfs.support.append</name>
>>                 <value>true</value>
>>         </property>
>>         <property>
>>                 <name>dfs.replication</name>
>>                 <value>2</value>
>>         </property>
>>
>>
>> Hope you can help us.
>> Thanks in advance.
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Thanks for reply, Nicolas.

My question: What can lead to shutdown of all of the datanodes?
I believe that the regionservers will be OK if the HDFS is OK.


On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Ok, what's your question?
> When you say the datanode went down, was it the datanode processes or the
> machines, with both the datanodes and the regionservers?
>
> The NameNode pings its datanodes every 3 seconds. However it will internally
> mark the datanodes as dead after 10:30 minutes (even if in the gui you have
> 'no answer for x minutes').
> HBase monitoring is done by ZooKeeper. By default, a regionserver is
> considered as dead after 180s with no answer. Before, well, it's considered
> as live.
> When you stop a regionserver, it tries to flush its data to the disk (i.e.
> hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a high
> ratio of your datanodes are dead, it can't shutdown. Connection refused &
> socket timeouts come from the fact that before the 10:30 minutes hdfs does
> not declare the nodes as dead, so hbase tries to use them (and, obviously,
> fails). Note that there is now  an intermediate state for hdfs datanodes,
> called "stale": an intermediary state where the datanode is used only if you
> have to (i.e. it's the only datanode with a block replica you need). It will
> be documented in HBase for the 0.96 release. But if all your datanodes are
> down it won't change much.
>
> Cheers,
>
> Nicolas
>
>
>
> On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Hey guys,
>>
>> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> year, and it works fine.
>> But the datanodes got shutdown twice in the last month.
>>
>> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> but regionservers of HBase were still live in the HBase web
>> admin(http://ip:60010/master-status), of course, they were zombies.
>> All of the processes of jvm were still running, including
>> hmaster/namenode/regionserver/datanode.
>>
>> When the datanodes got shutdown, the load (using the "top" command) of
>> slaves became very high, more than 10, higher than normal running.
>> From the "top" command, we saw that the processes of datanode and
>> regionserver were comsuming CPU.
>>
>> We could not stop the HBase or Hadoop cluster through normal
>> commands(stop-*.sh/*-daemon.sh stop *).
>> So we stopped datanodes and regionservers by kill -9 PID, then the
>> load of slaves returned to normal level, and we start the cluster
>> again.
>>
>>
>> Log of NN at the shutdown point(All of the DNs were removed):
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.152:50010
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.148:50010
>> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.148:50010
>>
>>
>> Logs in DNs indicated there were many IOException and
>> SocketTimeoutException:
>> 2013-02-22 11:02:52,354 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.io.IOException: Interrupted receiveBlock
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:03:44,823 WARN
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):Got exception while serving
>> blk_-1985405101514576650_247001 to /192.168.1.148:
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:48654]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:09:42,294 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:37188]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:12:41,892 INFO
>> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> succeeded for blk_-2674357249542194287_43419
>>
>>
>> Here is our env:
>> hadoop 1.0.3
>> hbase 0.94.1(snappy enabled)
>>
>> java version "1.6.0_31"
>> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>
>> # ulimit -a
>> core file size          (blocks, -c) 0
>> data seg size           (kbytes, -d) unlimited
>> scheduling priority             (-e) 20
>> file size               (blocks, -f) unlimited
>> pending signals                 (-i) 16382
>> max locked memory       (kbytes, -l) 64
>> max memory size         (kbytes, -m) unlimited
>> open files                      (-n) 32768
>> pipe size            (512 bytes, -p) 8
>> POSIX message queues     (bytes, -q) 819200
>> real-time priority              (-r) 0
>> stack size              (kbytes, -s) 8192
>> cpu time               (seconds, -t) unlimited
>> max user processes              (-u) 32768
>> virtual memory          (kbytes, -v) unlimited
>> file locks                      (-x) unlimited
>>
>> # uname -a
>> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> UTC 2011 x86_64 GNU/Linux
>>
>>
>> # free(master)
>>              total       used       free     shared    buffers     cached
>> Mem:      24732936    8383708   16349228          0     490584    2580356
>> -/+ buffers/cache:    5312768   19420168
>> Swap:     72458232          0   72458232
>>
>>
>> # free(slaves)
>>              total       used       free     shared    buffers     cached
>> Mem:      24733000   22824276    1908724          0     862556   15303304
>> -/+ buffers/cache:    6658416   18074584
>> Swap:     72458232        264   72457968
>>
>>
>> Some important conf:
>> core-site.xml
>>         <property>
>>                 <name>io.file.buffer.size</name>
>>                 <value>65536</value>
>>         </property>
>>
>> hdfs-site.xml
>>         <property>
>>                 <name>dfs.block.size</name>
>>                 <value>134217728</value>
>>         </property>
>>         <property>
>>                 <name>dfs.datanode.max.xcievers</name>
>>                 <value>4096</value>
>>         </property>
>>         <property>
>>                 <name>dfs.support.append</name>
>>                 <value>true</value>
>>         </property>
>>         <property>
>>                 <name>dfs.replication</name>
>>                 <value>2</value>
>>         </property>
>>
>>
>> Hope you can help us.
>> Thanks in advance.
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Thanks for reply, Nicolas.

My question: What can lead to shutdown of all of the datanodes?
I believe that the regionservers will be OK if the HDFS is OK.


On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Ok, what's your question?
> When you say the datanode went down, was it the datanode processes or the
> machines, with both the datanodes and the regionservers?
>
> The NameNode pings its datanodes every 3 seconds. However it will internally
> mark the datanodes as dead after 10:30 minutes (even if in the gui you have
> 'no answer for x minutes').
> HBase monitoring is done by ZooKeeper. By default, a regionserver is
> considered as dead after 180s with no answer. Before, well, it's considered
> as live.
> When you stop a regionserver, it tries to flush its data to the disk (i.e.
> hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a high
> ratio of your datanodes are dead, it can't shutdown. Connection refused &
> socket timeouts come from the fact that before the 10:30 minutes hdfs does
> not declare the nodes as dead, so hbase tries to use them (and, obviously,
> fails). Note that there is now  an intermediate state for hdfs datanodes,
> called "stale": an intermediary state where the datanode is used only if you
> have to (i.e. it's the only datanode with a block replica you need). It will
> be documented in HBase for the 0.96 release. But if all your datanodes are
> down it won't change much.
>
> Cheers,
>
> Nicolas
>
>
>
> On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Hey guys,
>>
>> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> year, and it works fine.
>> But the datanodes got shutdown twice in the last month.
>>
>> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> but regionservers of HBase were still live in the HBase web
>> admin(http://ip:60010/master-status), of course, they were zombies.
>> All of the processes of jvm were still running, including
>> hmaster/namenode/regionserver/datanode.
>>
>> When the datanodes got shutdown, the load (using the "top" command) of
>> slaves became very high, more than 10, higher than normal running.
>> From the "top" command, we saw that the processes of datanode and
>> regionserver were comsuming CPU.
>>
>> We could not stop the HBase or Hadoop cluster through normal
>> commands(stop-*.sh/*-daemon.sh stop *).
>> So we stopped datanodes and regionservers by kill -9 PID, then the
>> load of slaves returned to normal level, and we start the cluster
>> again.
>>
>>
>> Log of NN at the shutdown point(All of the DNs were removed):
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.152:50010
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.148:50010
>> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.148:50010
>>
>>
>> Logs in DNs indicated there were many IOException and
>> SocketTimeoutException:
>> 2013-02-22 11:02:52,354 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.io.IOException: Interrupted receiveBlock
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:03:44,823 WARN
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):Got exception while serving
>> blk_-1985405101514576650_247001 to /192.168.1.148:
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:48654]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:09:42,294 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:37188]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:12:41,892 INFO
>> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> succeeded for blk_-2674357249542194287_43419
>>
>>
>> Here is our env:
>> hadoop 1.0.3
>> hbase 0.94.1(snappy enabled)
>>
>> java version "1.6.0_31"
>> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>
>> # ulimit -a
>> core file size          (blocks, -c) 0
>> data seg size           (kbytes, -d) unlimited
>> scheduling priority             (-e) 20
>> file size               (blocks, -f) unlimited
>> pending signals                 (-i) 16382
>> max locked memory       (kbytes, -l) 64
>> max memory size         (kbytes, -m) unlimited
>> open files                      (-n) 32768
>> pipe size            (512 bytes, -p) 8
>> POSIX message queues     (bytes, -q) 819200
>> real-time priority              (-r) 0
>> stack size              (kbytes, -s) 8192
>> cpu time               (seconds, -t) unlimited
>> max user processes              (-u) 32768
>> virtual memory          (kbytes, -v) unlimited
>> file locks                      (-x) unlimited
>>
>> # uname -a
>> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> UTC 2011 x86_64 GNU/Linux
>>
>>
>> # free(master)
>>              total       used       free     shared    buffers     cached
>> Mem:      24732936    8383708   16349228          0     490584    2580356
>> -/+ buffers/cache:    5312768   19420168
>> Swap:     72458232          0   72458232
>>
>>
>> # free(slaves)
>>              total       used       free     shared    buffers     cached
>> Mem:      24733000   22824276    1908724          0     862556   15303304
>> -/+ buffers/cache:    6658416   18074584
>> Swap:     72458232        264   72457968
>>
>>
>> Some important conf:
>> core-site.xml
>>         <property>
>>                 <name>io.file.buffer.size</name>
>>                 <value>65536</value>
>>         </property>
>>
>> hdfs-site.xml
>>         <property>
>>                 <name>dfs.block.size</name>
>>                 <value>134217728</value>
>>         </property>
>>         <property>
>>                 <name>dfs.datanode.max.xcievers</name>
>>                 <value>4096</value>
>>         </property>
>>         <property>
>>                 <name>dfs.support.append</name>
>>                 <value>true</value>
>>         </property>
>>         <property>
>>                 <name>dfs.replication</name>
>>                 <value>2</value>
>>         </property>
>>
>>
>> Hope you can help us.
>> Thanks in advance.
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Davey Yan <da...@gmail.com>.
Thanks for reply, Nicolas.

My question: What can lead to shutdown of all of the datanodes?
I believe that the regionservers will be OK if the HDFS is OK.


On Mon, Feb 25, 2013 at 5:31 PM, Nicolas Liochon <nk...@gmail.com> wrote:
> Ok, what's your question?
> When you say the datanode went down, was it the datanode processes or the
> machines, with both the datanodes and the regionservers?
>
> The NameNode pings its datanodes every 3 seconds. However it will internally
> mark the datanodes as dead after 10:30 minutes (even if in the gui you have
> 'no answer for x minutes').
> HBase monitoring is done by ZooKeeper. By default, a regionserver is
> considered as dead after 180s with no answer. Before, well, it's considered
> as live.
> When you stop a regionserver, it tries to flush its data to the disk (i.e.
> hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a high
> ratio of your datanodes are dead, it can't shutdown. Connection refused &
> socket timeouts come from the fact that before the 10:30 minutes hdfs does
> not declare the nodes as dead, so hbase tries to use them (and, obviously,
> fails). Note that there is now  an intermediate state for hdfs datanodes,
> called "stale": an intermediary state where the datanode is used only if you
> have to (i.e. it's the only datanode with a block replica you need). It will
> be documented in HBase for the 0.96 release. But if all your datanodes are
> down it won't change much.
>
> Cheers,
>
> Nicolas
>
>
>
> On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:
>>
>> Hey guys,
>>
>> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
>> year, and it works fine.
>> But the datanodes got shutdown twice in the last month.
>>
>> When the datanodes got shutdown, all of them became "Dead Nodes" in
>> the NN web admin UI(http://ip:50070/dfshealth.jsp),
>> but regionservers of HBase were still live in the HBase web
>> admin(http://ip:60010/master-status), of course, they were zombies.
>> All of the processes of jvm were still running, including
>> hmaster/namenode/regionserver/datanode.
>>
>> When the datanodes got shutdown, the load (using the "top" command) of
>> slaves became very high, more than 10, higher than normal running.
>> From the "top" command, we saw that the processes of datanode and
>> regionserver were comsuming CPU.
>>
>> We could not stop the HBase or Hadoop cluster through normal
>> commands(stop-*.sh/*-daemon.sh stop *).
>> So we stopped datanodes and regionservers by kill -9 PID, then the
>> load of slaves returned to normal level, and we start the cluster
>> again.
>>
>>
>> Log of NN at the shutdown point(All of the DNs were removed):
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.152:50010
>> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.149:50010
>> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.150:50010
>> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
>> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
>> 192.168.1.148:50010
>> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
>> Removing a node: /default-rack/192.168.1.148:50010
>>
>>
>> Logs in DNs indicated there were many IOException and
>> SocketTimeoutException:
>> 2013-02-22 11:02:52,354 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.io.IOException: Interrupted receiveBlock
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:03:44,823 WARN
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):Got exception while serving
>> blk_-1985405101514576650_247001 to /192.168.1.148:
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:48654]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:09:42,294 ERROR
>> org.apache.hadoop.hdfs.server.datanode.DataNode:
>> DatanodeRegistration(192.168.1.148:50010,
>> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
>> infoPort=50075, ipcPort=50020):DataXceiver
>> java.net.SocketTimeoutException: 480000 millis timeout while waiting
>> for channel to be ready for write. ch :
>> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
>> remote=/192.168.1.148:37188]
>>         at
>> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>>         at
>> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>>         at
>> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>>         at java.lang.Thread.run(Thread.java:662)
>> 2013-02-22 11:12:41,892 INFO
>> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
>> succeeded for blk_-2674357249542194287_43419
>>
>>
>> Here is our env:
>> hadoop 1.0.3
>> hbase 0.94.1(snappy enabled)
>>
>> java version "1.6.0_31"
>> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
>> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>>
>> # ulimit -a
>> core file size          (blocks, -c) 0
>> data seg size           (kbytes, -d) unlimited
>> scheduling priority             (-e) 20
>> file size               (blocks, -f) unlimited
>> pending signals                 (-i) 16382
>> max locked memory       (kbytes, -l) 64
>> max memory size         (kbytes, -m) unlimited
>> open files                      (-n) 32768
>> pipe size            (512 bytes, -p) 8
>> POSIX message queues     (bytes, -q) 819200
>> real-time priority              (-r) 0
>> stack size              (kbytes, -s) 8192
>> cpu time               (seconds, -t) unlimited
>> max user processes              (-u) 32768
>> virtual memory          (kbytes, -v) unlimited
>> file locks                      (-x) unlimited
>>
>> # uname -a
>> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
>> UTC 2011 x86_64 GNU/Linux
>>
>>
>> # free(master)
>>              total       used       free     shared    buffers     cached
>> Mem:      24732936    8383708   16349228          0     490584    2580356
>> -/+ buffers/cache:    5312768   19420168
>> Swap:     72458232          0   72458232
>>
>>
>> # free(slaves)
>>              total       used       free     shared    buffers     cached
>> Mem:      24733000   22824276    1908724          0     862556   15303304
>> -/+ buffers/cache:    6658416   18074584
>> Swap:     72458232        264   72457968
>>
>>
>> Some important conf:
>> core-site.xml
>>         <property>
>>                 <name>io.file.buffer.size</name>
>>                 <value>65536</value>
>>         </property>
>>
>> hdfs-site.xml
>>         <property>
>>                 <name>dfs.block.size</name>
>>                 <value>134217728</value>
>>         </property>
>>         <property>
>>                 <name>dfs.datanode.max.xcievers</name>
>>                 <value>4096</value>
>>         </property>
>>         <property>
>>                 <name>dfs.support.append</name>
>>                 <value>true</value>
>>         </property>
>>         <property>
>>                 <name>dfs.replication</name>
>>                 <value>2</value>
>>         </property>
>>
>>
>> Hope you can help us.
>> Thanks in advance.
>>
>>
>>
>> --
>> Davey Yan
>
>



-- 
Davey Yan

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Ok, what's your question?
When you say the datanode went down, was it the datanode processes or the
machines, with both the datanodes and the regionservers?

The NameNode pings its datanodes every 3 seconds. However it will
internally mark the datanodes as dead after 10:30 minutes (even if in the
gui you have 'no answer for x minutes').
HBase monitoring is done by ZooKeeper. By default, a regionserver is
considered as dead after 180s with no answer. Before, well, it's considered
as live.
When you stop a regionserver, it tries to flush its data to the disk (i.e.
hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
high ratio of your datanodes are dead, it can't shutdown. Connection
refused & socket timeouts come from the fact that before the 10:30 minutes
hdfs does not declare the nodes as dead, so hbase tries to use them (and,
obviously, fails). Note that there is now  an intermediate state for hdfs
datanodes, called "stale": an intermediary state where the datanode is used
only if you have to (i.e. it's the only datanode with a block replica you
need). It will be documented in HBase for the 0.96 release. But if all your
datanodes are down it won't change much.

Cheers,

Nicolas



On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:

> Hey guys,
>
> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> year, and it works fine.
> But the datanodes got shutdown twice in the last month.
>
> When the datanodes got shutdown, all of them became "Dead Nodes" in
> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> but regionservers of HBase were still live in the HBase web
> admin(http://ip:60010/master-status), of course, they were zombies.
> All of the processes of jvm were still running, including
> hmaster/namenode/regionserver/datanode.
>
> When the datanodes got shutdown, the load (using the "top" command) of
> slaves became very high, more than 10, higher than normal running.
> From the "top" command, we saw that the processes of datanode and
> regionserver were comsuming CPU.
>
> We could not stop the HBase or Hadoop cluster through normal
> commands(stop-*.sh/*-daemon.sh stop *).
> So we stopped datanodes and regionservers by kill -9 PID, then the
> load of slaves returned to normal level, and we start the cluster
> again.
>
>
> Log of NN at the shutdown point(All of the DNs were removed):
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.152:50010
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.148:50010
> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.148:50010
>
>
> Logs in DNs indicated there were many IOException and
> SocketTimeoutException:
> 2013-02-22 11:02:52,354 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Interrupted receiveBlock
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:03:44,823 WARN
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):Got exception while serving
> blk_-1985405101514576650_247001 to /192.168.1.148:
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:48654]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:09:42,294 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:37188]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:12:41,892 INFO
> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> succeeded for blk_-2674357249542194287_43419
>
>
> Here is our env:
> hadoop 1.0.3
> hbase 0.94.1(snappy enabled)
>
> java version "1.6.0_31"
> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>
> # ulimit -a
> core file size          (blocks, -c) 0
> data seg size           (kbytes, -d) unlimited
> scheduling priority             (-e) 20
> file size               (blocks, -f) unlimited
> pending signals                 (-i) 16382
> max locked memory       (kbytes, -l) 64
> max memory size         (kbytes, -m) unlimited
> open files                      (-n) 32768
> pipe size            (512 bytes, -p) 8
> POSIX message queues     (bytes, -q) 819200
> real-time priority              (-r) 0
> stack size              (kbytes, -s) 8192
> cpu time               (seconds, -t) unlimited
> max user processes              (-u) 32768
> virtual memory          (kbytes, -v) unlimited
> file locks                      (-x) unlimited
>
> # uname -a
> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> UTC 2011 x86_64 GNU/Linux
>
>
> # free(master)
>              total       used       free     shared    buffers     cached
> Mem:      24732936    8383708   16349228          0     490584    2580356
> -/+ buffers/cache:    5312768   19420168
> Swap:     72458232          0   72458232
>
>
> # free(slaves)
>              total       used       free     shared    buffers     cached
> Mem:      24733000   22824276    1908724          0     862556   15303304
> -/+ buffers/cache:    6658416   18074584
> Swap:     72458232        264   72457968
>
>
> Some important conf:
> core-site.xml
>         <property>
>                 <name>io.file.buffer.size</name>
>                 <value>65536</value>
>         </property>
>
> hdfs-site.xml
>         <property>
>                 <name>dfs.block.size</name>
>                 <value>134217728</value>
>         </property>
>         <property>
>                 <name>dfs.datanode.max.xcievers</name>
>                 <value>4096</value>
>         </property>
>         <property>
>                 <name>dfs.support.append</name>
>                 <value>true</value>
>         </property>
>         <property>
>                 <name>dfs.replication</name>
>                 <value>2</value>
>         </property>
>
>
> Hope you can help us.
> Thanks in advance.
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Ok, what's your question?
When you say the datanode went down, was it the datanode processes or the
machines, with both the datanodes and the regionservers?

The NameNode pings its datanodes every 3 seconds. However it will
internally mark the datanodes as dead after 10:30 minutes (even if in the
gui you have 'no answer for x minutes').
HBase monitoring is done by ZooKeeper. By default, a regionserver is
considered as dead after 180s with no answer. Before, well, it's considered
as live.
When you stop a regionserver, it tries to flush its data to the disk (i.e.
hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
high ratio of your datanodes are dead, it can't shutdown. Connection
refused & socket timeouts come from the fact that before the 10:30 minutes
hdfs does not declare the nodes as dead, so hbase tries to use them (and,
obviously, fails). Note that there is now  an intermediate state for hdfs
datanodes, called "stale": an intermediary state where the datanode is used
only if you have to (i.e. it's the only datanode with a block replica you
need). It will be documented in HBase for the 0.96 release. But if all your
datanodes are down it won't change much.

Cheers,

Nicolas



On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:

> Hey guys,
>
> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> year, and it works fine.
> But the datanodes got shutdown twice in the last month.
>
> When the datanodes got shutdown, all of them became "Dead Nodes" in
> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> but regionservers of HBase were still live in the HBase web
> admin(http://ip:60010/master-status), of course, they were zombies.
> All of the processes of jvm were still running, including
> hmaster/namenode/regionserver/datanode.
>
> When the datanodes got shutdown, the load (using the "top" command) of
> slaves became very high, more than 10, higher than normal running.
> From the "top" command, we saw that the processes of datanode and
> regionserver were comsuming CPU.
>
> We could not stop the HBase or Hadoop cluster through normal
> commands(stop-*.sh/*-daemon.sh stop *).
> So we stopped datanodes and regionservers by kill -9 PID, then the
> load of slaves returned to normal level, and we start the cluster
> again.
>
>
> Log of NN at the shutdown point(All of the DNs were removed):
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.152:50010
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.148:50010
> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.148:50010
>
>
> Logs in DNs indicated there were many IOException and
> SocketTimeoutException:
> 2013-02-22 11:02:52,354 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Interrupted receiveBlock
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:03:44,823 WARN
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):Got exception while serving
> blk_-1985405101514576650_247001 to /192.168.1.148:
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:48654]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:09:42,294 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:37188]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:12:41,892 INFO
> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> succeeded for blk_-2674357249542194287_43419
>
>
> Here is our env:
> hadoop 1.0.3
> hbase 0.94.1(snappy enabled)
>
> java version "1.6.0_31"
> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>
> # ulimit -a
> core file size          (blocks, -c) 0
> data seg size           (kbytes, -d) unlimited
> scheduling priority             (-e) 20
> file size               (blocks, -f) unlimited
> pending signals                 (-i) 16382
> max locked memory       (kbytes, -l) 64
> max memory size         (kbytes, -m) unlimited
> open files                      (-n) 32768
> pipe size            (512 bytes, -p) 8
> POSIX message queues     (bytes, -q) 819200
> real-time priority              (-r) 0
> stack size              (kbytes, -s) 8192
> cpu time               (seconds, -t) unlimited
> max user processes              (-u) 32768
> virtual memory          (kbytes, -v) unlimited
> file locks                      (-x) unlimited
>
> # uname -a
> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> UTC 2011 x86_64 GNU/Linux
>
>
> # free(master)
>              total       used       free     shared    buffers     cached
> Mem:      24732936    8383708   16349228          0     490584    2580356
> -/+ buffers/cache:    5312768   19420168
> Swap:     72458232          0   72458232
>
>
> # free(slaves)
>              total       used       free     shared    buffers     cached
> Mem:      24733000   22824276    1908724          0     862556   15303304
> -/+ buffers/cache:    6658416   18074584
> Swap:     72458232        264   72457968
>
>
> Some important conf:
> core-site.xml
>         <property>
>                 <name>io.file.buffer.size</name>
>                 <value>65536</value>
>         </property>
>
> hdfs-site.xml
>         <property>
>                 <name>dfs.block.size</name>
>                 <value>134217728</value>
>         </property>
>         <property>
>                 <name>dfs.datanode.max.xcievers</name>
>                 <value>4096</value>
>         </property>
>         <property>
>                 <name>dfs.support.append</name>
>                 <value>true</value>
>         </property>
>         <property>
>                 <name>dfs.replication</name>
>                 <value>2</value>
>         </property>
>
>
> Hope you can help us.
> Thanks in advance.
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Ok, what's your question?
When you say the datanode went down, was it the datanode processes or the
machines, with both the datanodes and the regionservers?

The NameNode pings its datanodes every 3 seconds. However it will
internally mark the datanodes as dead after 10:30 minutes (even if in the
gui you have 'no answer for x minutes').
HBase monitoring is done by ZooKeeper. By default, a regionserver is
considered as dead after 180s with no answer. Before, well, it's considered
as live.
When you stop a regionserver, it tries to flush its data to the disk (i.e.
hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
high ratio of your datanodes are dead, it can't shutdown. Connection
refused & socket timeouts come from the fact that before the 10:30 minutes
hdfs does not declare the nodes as dead, so hbase tries to use them (and,
obviously, fails). Note that there is now  an intermediate state for hdfs
datanodes, called "stale": an intermediary state where the datanode is used
only if you have to (i.e. it's the only datanode with a block replica you
need). It will be documented in HBase for the 0.96 release. But if all your
datanodes are down it won't change much.

Cheers,

Nicolas



On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:

> Hey guys,
>
> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> year, and it works fine.
> But the datanodes got shutdown twice in the last month.
>
> When the datanodes got shutdown, all of them became "Dead Nodes" in
> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> but regionservers of HBase were still live in the HBase web
> admin(http://ip:60010/master-status), of course, they were zombies.
> All of the processes of jvm were still running, including
> hmaster/namenode/regionserver/datanode.
>
> When the datanodes got shutdown, the load (using the "top" command) of
> slaves became very high, more than 10, higher than normal running.
> From the "top" command, we saw that the processes of datanode and
> regionserver were comsuming CPU.
>
> We could not stop the HBase or Hadoop cluster through normal
> commands(stop-*.sh/*-daemon.sh stop *).
> So we stopped datanodes and regionservers by kill -9 PID, then the
> load of slaves returned to normal level, and we start the cluster
> again.
>
>
> Log of NN at the shutdown point(All of the DNs were removed):
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.152:50010
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.148:50010
> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.148:50010
>
>
> Logs in DNs indicated there were many IOException and
> SocketTimeoutException:
> 2013-02-22 11:02:52,354 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Interrupted receiveBlock
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:03:44,823 WARN
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):Got exception while serving
> blk_-1985405101514576650_247001 to /192.168.1.148:
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:48654]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:09:42,294 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:37188]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:12:41,892 INFO
> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> succeeded for blk_-2674357249542194287_43419
>
>
> Here is our env:
> hadoop 1.0.3
> hbase 0.94.1(snappy enabled)
>
> java version "1.6.0_31"
> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>
> # ulimit -a
> core file size          (blocks, -c) 0
> data seg size           (kbytes, -d) unlimited
> scheduling priority             (-e) 20
> file size               (blocks, -f) unlimited
> pending signals                 (-i) 16382
> max locked memory       (kbytes, -l) 64
> max memory size         (kbytes, -m) unlimited
> open files                      (-n) 32768
> pipe size            (512 bytes, -p) 8
> POSIX message queues     (bytes, -q) 819200
> real-time priority              (-r) 0
> stack size              (kbytes, -s) 8192
> cpu time               (seconds, -t) unlimited
> max user processes              (-u) 32768
> virtual memory          (kbytes, -v) unlimited
> file locks                      (-x) unlimited
>
> # uname -a
> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> UTC 2011 x86_64 GNU/Linux
>
>
> # free(master)
>              total       used       free     shared    buffers     cached
> Mem:      24732936    8383708   16349228          0     490584    2580356
> -/+ buffers/cache:    5312768   19420168
> Swap:     72458232          0   72458232
>
>
> # free(slaves)
>              total       used       free     shared    buffers     cached
> Mem:      24733000   22824276    1908724          0     862556   15303304
> -/+ buffers/cache:    6658416   18074584
> Swap:     72458232        264   72457968
>
>
> Some important conf:
> core-site.xml
>         <property>
>                 <name>io.file.buffer.size</name>
>                 <value>65536</value>
>         </property>
>
> hdfs-site.xml
>         <property>
>                 <name>dfs.block.size</name>
>                 <value>134217728</value>
>         </property>
>         <property>
>                 <name>dfs.datanode.max.xcievers</name>
>                 <value>4096</value>
>         </property>
>         <property>
>                 <name>dfs.support.append</name>
>                 <value>true</value>
>         </property>
>         <property>
>                 <name>dfs.replication</name>
>                 <value>2</value>
>         </property>
>
>
> Hope you can help us.
> Thanks in advance.
>
>
>
> --
> Davey Yan
>

Re: Datanodes shutdown and HBase's regionservers not working

Posted by Nicolas Liochon <nk...@gmail.com>.
Ok, what's your question?
When you say the datanode went down, was it the datanode processes or the
machines, with both the datanodes and the regionservers?

The NameNode pings its datanodes every 3 seconds. However it will
internally mark the datanodes as dead after 10:30 minutes (even if in the
gui you have 'no answer for x minutes').
HBase monitoring is done by ZooKeeper. By default, a regionserver is
considered as dead after 180s with no answer. Before, well, it's considered
as live.
When you stop a regionserver, it tries to flush its data to the disk (i.e.
hdfs, i.e. the datanodes). That's why if you have no datanodes, or if a
high ratio of your datanodes are dead, it can't shutdown. Connection
refused & socket timeouts come from the fact that before the 10:30 minutes
hdfs does not declare the nodes as dead, so hbase tries to use them (and,
obviously, fails). Note that there is now  an intermediate state for hdfs
datanodes, called "stale": an intermediary state where the datanode is used
only if you have to (i.e. it's the only datanode with a block replica you
need). It will be documented in HBase for the 0.96 release. But if all your
datanodes are down it won't change much.

Cheers,

Nicolas



On Mon, Feb 25, 2013 at 10:10 AM, Davey Yan <da...@gmail.com> wrote:

> Hey guys,
>
> We have a cluster with 5 nodes(1 NN and 4 DNs) running for more than 1
> year, and it works fine.
> But the datanodes got shutdown twice in the last month.
>
> When the datanodes got shutdown, all of them became "Dead Nodes" in
> the NN web admin UI(http://ip:50070/dfshealth.jsp),
> but regionservers of HBase were still live in the HBase web
> admin(http://ip:60010/master-status), of course, they were zombies.
> All of the processes of jvm were still running, including
> hmaster/namenode/regionserver/datanode.
>
> When the datanodes got shutdown, the load (using the "top" command) of
> slaves became very high, more than 10, higher than normal running.
> From the "top" command, we saw that the processes of datanode and
> regionserver were comsuming CPU.
>
> We could not stop the HBase or Hadoop cluster through normal
> commands(stop-*.sh/*-daemon.sh stop *).
> So we stopped datanodes and regionservers by kill -9 PID, then the
> load of slaves returned to normal level, and we start the cluster
> again.
>
>
> Log of NN at the shutdown point(All of the DNs were removed):
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.152:50010
> 2013-02-22 11:10:02,278 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.149:50010
> 2013-02-22 11:10:02,693 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.150:50010
> 2013-02-22 11:10:03,004 INFO org.apache.hadoop.hdfs.StateChange:
> BLOCK* NameSystem.heartbeatCheck: lost heartbeat from
> 192.168.1.148:50010
> 2013-02-22 11:10:03,339 INFO org.apache.hadoop.net.NetworkTopology:
> Removing a node: /default-rack/192.168.1.148:50010
>
>
> Logs in DNs indicated there were many IOException and
> SocketTimeoutException:
> 2013-02-22 11:02:52,354 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.io.IOException: Interrupted receiveBlock
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:577)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:398)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:107)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:03:44,823 WARN
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):Got exception while serving
> blk_-1985405101514576650_247001 to /192.168.1.148:
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:48654]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:09:42,294 ERROR
> org.apache.hadoop.hdfs.server.datanode.DataNode:
> DatanodeRegistration(192.168.1.148:50010,
> storageID=DS-970284113-117.25.149.160-50010-1328074119937,
> infoPort=50075, ipcPort=50020):DataXceiver
> java.net.SocketTimeoutException: 480000 millis timeout while waiting
> for channel to be ready for write. ch :
> java.nio.channels.SocketChannel[connected local=/192.168.1.148:50010
> remote=/192.168.1.148:37188]
>         at
> org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeout.java:246)
>         at
> org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutputStream.java:159)
>         at
> org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:198)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>         at
> org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:197)
>         at
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:99)
>         at java.lang.Thread.run(Thread.java:662)
> 2013-02-22 11:12:41,892 INFO
> org.apache.hadoop.hdfs.server.datanode.DataBlockScanner: Verification
> succeeded for blk_-2674357249542194287_43419
>
>
> Here is our env:
> hadoop 1.0.3
> hbase 0.94.1(snappy enabled)
>
> java version "1.6.0_31"
> Java(TM) SE Runtime Environment (build 1.6.0_31-b04)
> Java HotSpot(TM) 64-Bit Server VM (build 20.6-b01, mixed mode)
>
> # ulimit -a
> core file size          (blocks, -c) 0
> data seg size           (kbytes, -d) unlimited
> scheduling priority             (-e) 20
> file size               (blocks, -f) unlimited
> pending signals                 (-i) 16382
> max locked memory       (kbytes, -l) 64
> max memory size         (kbytes, -m) unlimited
> open files                      (-n) 32768
> pipe size            (512 bytes, -p) 8
> POSIX message queues     (bytes, -q) 819200
> real-time priority              (-r) 0
> stack size              (kbytes, -s) 8192
> cpu time               (seconds, -t) unlimited
> max user processes              (-u) 32768
> virtual memory          (kbytes, -v) unlimited
> file locks                      (-x) unlimited
>
> # uname -a
> Linux ubuntu6401 2.6.32-33-server #70-Ubuntu SMP Thu Jul 7 22:28:30
> UTC 2011 x86_64 GNU/Linux
>
>
> # free(master)
>              total       used       free     shared    buffers     cached
> Mem:      24732936    8383708   16349228          0     490584    2580356
> -/+ buffers/cache:    5312768   19420168
> Swap:     72458232          0   72458232
>
>
> # free(slaves)
>              total       used       free     shared    buffers     cached
> Mem:      24733000   22824276    1908724          0     862556   15303304
> -/+ buffers/cache:    6658416   18074584
> Swap:     72458232        264   72457968
>
>
> Some important conf:
> core-site.xml
>         <property>
>                 <name>io.file.buffer.size</name>
>                 <value>65536</value>
>         </property>
>
> hdfs-site.xml
>         <property>
>                 <name>dfs.block.size</name>
>                 <value>134217728</value>
>         </property>
>         <property>
>                 <name>dfs.datanode.max.xcievers</name>
>                 <value>4096</value>
>         </property>
>         <property>
>                 <name>dfs.support.append</name>
>                 <value>true</value>
>         </property>
>         <property>
>                 <name>dfs.replication</name>
>                 <value>2</value>
>         </property>
>
>
> Hope you can help us.
> Thanks in advance.
>
>
>
> --
> Davey Yan
>