Regards the timeout, its a configuration for the dfsclient, the thing hbase
uses communicating with hdfs.
./org/apache/hadoop/hdfs/DFSClient.java: this.datanodeWriteTimeout =
conf.getInt("dfs.datanode.socket.write.timeout",
If the config. is off in hadoop, hbase doesn't see it.
To get number of regions, you could do as you are doing -- counting table
subdirs -- but you could also just look in the UI (on port 60010 by
default).
A faster means of counting rows is to run a MR job to do it. HBase bundles
one called rowcounter.
Do something like the following presuming you have a little mapreduce
cluster up and poised:
./bin/hadoop -jar $HBASE_HOME/hbase*jar
This will list options (I think it throws an exception but in the exception
is says you need to supply a job name).
The job is rowcounter IIRC so you'd do:
./bin/hadoop -jar $HBASE_HOME/hbase*jar rowcounter
This will launch a MR job. At the end as part of the MR stats it prints
number of rows among other job stats.
If you continue to see errors, post another log snippet please for us to
take a look at (Include the bit where the regionserver starts logging
because it dumps out info on jvm etc. which can be useful).
Thanks,
St.Ack
On Wed, Feb 25, 2009 at 2:56 PM, Larry Compton
<[email protected]>wrote:
> Actually, I spoke too soon. In "hbase-env.xml", we have "HBASE_CLASSPATH"
> set to include the Hadoop conf directory on all 4 nodes, so the Hbase
> servers should have access to all of the Hadoop parameters. I'm going to
> try
> a symlink to "hadoop-site.xml" and see if the behavior changes.
>
> Larry
>
> On Wed, Feb 25, 2009 at 5:33 PM, Larry Compton
> <[email protected]>wrote:
>
> > "dfs.datanode.socket.write.timeout" is set in "hadoop-site.xml" and isn't
> > linked or contained in the Hbase "conf" directory. I'll try that out. I'm
> > not sure I understand why this is necessary, though. It seems like this
> > parameter would only matter to Hadoop, so why is it necessary for the
> Hbase
> > servers to have access to it?
> >
> > Also, I've been looking at the Hbase Wiki and also at the content stored
> in
> > my Hbase directory in HDFS. I can easily get the size in bytes of my
> table
> > using "hadoop fs -dus", but I don't know how to get the number of
> regions.
> > Are the regions the subdirectories directly beneath the table directory?
> > Also, what's a fast way to find out the number of rows? I've been trying
> to
> > use "count" in "hbase shell", but I keep getting scanner timeouts.
> >
> >
> > On Sat, Feb 21, 2009 at 12:45 AM, stack <[email protected]> wrote:
> >
> >> On Fri, Feb 20, 2009 at 2:49 PM, Larry Compton
> >> <[email protected]>wrote:
> >>
> >> > I'm having problems with my region servers dying. Region server and
> data
> >> > node log snippets are found below. Here's a synopsis of my
> >> configuration...
> >> > - 4 nodes
> >> > - Hadoop/Hbase 0.19.0
> >> > - dfs.datanode.max.xcievers - 2048
> >> > - dfs.datanode.socket.write.timeout - 0
> >> > - file handle limit - 32768
> >> > - fsck - healthy
> >>
> >>
> >> Thanks for reporting that you have above configured. What size table,
> >> regions and rows?
> >>
> >> Is the dfs.datanode.socket.write.timeout=0 set in a context that hbase
> can
> >> see it? i.e. is it in hbase-site or is it in hadoop-site and symlinked
> >> under the hbase/conf dir so hbase picks it up? Going by errors below,
> its
> >> absence could be explaination.
> >>
> >> Yours,
> >> St.Ack
> >>
> >>
> >> >
> >> > I'm seeing DataXceiver errors in the data node log, but not the sort
> >> that
> >> > indicates that the max.xcievers value is too small. Any idea what
> might
> >> be
> >> > wrong?
> >> >
> >> > HBASE REGION SERVER LOG OUTPUT...
> >> > 2009-02-20 08:50:42,476 WARN org.apache.hadoop.hdfs.DFSClient:
> >> DataStreamer
> >> > Exception: java.net.SocketTimeoutException: 5000 millis timeout while
> >> > waiting for channel to be ready for write. ch :
> >> > java.nio.channels.SocketChannel[connected local=/192.168.6.38:56737
> >> remote=/
> >> > 192.168.6.38:50010]
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:162)
> >> > at
> >> >
> >>
> org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146)
> >> > at
> >> >
> >>
> org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107)
> >> > at
> >> java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
> >> > at java.io.DataOutputStream.write(DataOutputStream.java:90)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2209)
> >> >
> >> > 2009-02-20 08:50:42,918 WARN org.apache.hadoop.hdfs.DFSClient:
> >> DataStreamer
> >> > Exception: java.net.SocketTimeoutException: 5000 millis timeout while
> >> > waiting for channel to be ready for write. ch :
> >> > java.nio.channels.SocketChannel[connected local=/192.168.6.38:56646
> >> remote=/
> >> > 192.168.6.38:50010]
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:162)
> >> > at
> >> >
> >>
> org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146)
> >> > at
> >> >
> >>
> org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107)
> >> > at
> >> java.io.BufferedOutputStream.write(BufferedOutputStream.java:105)
> >> > at java.io.DataOutputStream.write(DataOutputStream.java:90)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2209)
> >> >
> >> > 2009-02-20 08:50:43,023 WARN org.apache.hadoop.hdfs.DFSClient: Error
> >> > Recovery for block blk_2604922956617757726_298427 bad datanode[0]
> >> > 192.168.6.38:50010
> >> > 2009-02-20 08:50:43,023 WARN org.apache.hadoop.hdfs.DFSClient: Error
> >> > Recovery for block blk_-3747640666687562371_298377 bad datanode[0]
> >> > 192.168.6.38:50010
> >> > 2009-02-20 08:50:44,356 FATAL
> org.apache.hadoop.hbase.regionserver.HLog:
> >> > Could not append. Requesting close of log
> >> > java.io.IOException: All datanodes 192.168.6.38:50010 are bad.
> >> Aborting...
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2442)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$1600(DFSClient.java:1997)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2160)
> >> > 2009-02-20 08:50:44,357 ERROR
> >> > org.apache.hadoop.hbase.regionserver.CompactSplitThread:
> >> Compaction/Split
> >> > failed for region
> >> > medline,_X2dX5031454eX3aX11f48751c5eX3aXX2dX725c,1235136902878
> >> > java.io.IOException: All datanodes 192.168.6.38:50010 are bad.
> >> Aborting...
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2442)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$1600(DFSClient.java:1997)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2160)
> >> > 2009-02-20 08:50:44,377 ERROR
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer:
> java.io.IOException:
> >> > All
> >> > datanodes 192.168.6.38:50010 are bad. Aborting...
> >> > 2009-02-20 08:50:44,377 FATAL
> >> > org.apache.hadoop.hbase.regionserver.LogRoller: Log rolling failed
> with
> >> > ioe:
> >> > java.io.IOException: All datanodes 192.168.6.38:50010 are bad.
> >> Aborting...
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2442)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$1600(DFSClient.java:1997)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2160)
> >> > 2009-02-20 08:50:44,378 INFO
> >> org.apache.hadoop.hbase.regionserver.HRegion:
> >> > starting compaction on region medline,"blood",1235125955035
> >> > 2009-02-20 08:50:44,380 INFO org.apache.hadoop.ipc.HBaseServer: IPC
> >> Server
> >> > handler 6 on 60020, call batchUpdates([...@ecb0da,
> >> > [Lorg.apache.hadoop.hbase.io.BatchUpdate;@14ed87c) from
> >> 192.168.6.29:47457
> >> > :
> >> > error: java.io.IOException: All datanodes 192.168.6.38:50010 are bad.
> >> > Aborting...
> >> > java.io.IOException: All datanodes 192.168.6.38:50010 are bad.
> >> Aborting...
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:2442)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$1600(DFSClient.java:1997)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2160)
> >> > 2009-02-20 08:50:44,418 INFO
> >> > org.apache.hadoop.hbase.regionserver.HRegionServer: Dump of metrics:
> >> > request=2581, regions=71, stores=212, storefiles=352,
> >> > storefileIndexSize=31,
> >> > memcacheSize=574, usedHeap=1190, maxHeap=1984
> >> >
> >> > DATANODE LOG OUTPUT...
> >> > 2009-02-20 08:50:45,337 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder
> >> > blk_-3747640666687562371_298377 0 Exception java.net.SocketException:
> >> > Broken
> >> > pipe
> >> > at java.net.SocketOutputStream.socketWrite0(Native Method)
> >> > at
> >> > java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
> >> > at
> java.net.SocketOutputStream.write(SocketOutputStream.java:136)
> >> > at
> java.io.DataOutputStream.writeLong(DataOutputStream.java:207)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.lastDataNodeRun(BlockReceiver.java:797)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:820)
> >> > at java.lang.Thread.run(Thread.java:619)
> >> >
> >> > 2009-02-20 08:50:45,337 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for
> >> > block
> >> > blk_-3747640666687562371_298377 terminating
> >> > 2009-02-20 08:50:45,337 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock
> >> > blk_-3747640666687562371_298377 received exception
> java.io.EOFException:
> >> > while trying to read 32873 bytes
> >> > 2009-02-20 08:50:45,337 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder
> >> > blk_2604922956617757726_298427 0 Exception java.net.SocketException:
> >> Broken
> >> > pipe
> >> > at java.net.SocketOutputStream.socketWrite0(Native Method)
> >> > at
> >> > java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
> >> > at
> java.net.SocketOutputStream.write(SocketOutputStream.java:115)
> >> > at
> java.io.DataOutputStream.writeShort(DataOutputStream.java:150)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.lastDataNodeRun(BlockReceiver.java:798)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:820)
> >> > at java.lang.Thread.run(Thread.java:619)
> >> >
> >> > 2009-02-20 08:50:45,338 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 0 for
> >> > block
> >> > blk_2604922956617757726_298427 terminating
> >> > 2009-02-20 08:50:45,338 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock
> >> > blk_2604922956617757726_298427 received exception
> java.io.EOFException:
> >> > while trying to read 49299 bytes
> >> > 2009-02-20 08:50:45,342 INFO
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace: src: /
> >> > 192.168.6.38:50010, dest: /192.168.6.38:56791, bytes: 3318, op:
> >> HDFS_READ,
> >> > cliID: DFSClient_1697856093, srvID:
> >> > DS-697440498-192.168.6.38-50010-1233008986086, blockid:
> >> > blk_-4029959142608094898_296648
> >> > 2009-02-20 08:50:46,680 ERROR
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(
> >> > 192.168.6.38:50010,
> >> > storageID=DS-697440498-192.168.6.38-50010-1233008986086,
> >> > infoPort=50075, ipcPort=50020):DataXceiver
> >> > java.io.EOFException: while trying to read 32873 bytes
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readToBuf(BlockReceiver.java:254)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readNextPacket(BlockReceiver.java:341)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:362)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:514)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:356)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:102)
> >> > at java.lang.Thread.run(Thread.java:619)
> >> > 2009-02-20 08:50:46,680 ERROR
> >> > org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(
> >> > 192.168.6.38:50010,
> >> > storageID=DS-697440498-192.168.6.38-50010-1233008986086,
> >> > infoPort=50075, ipcPort=50020):DataXceiver
> >> > java.io.EOFException: while trying to read 49299 bytes
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readToBuf(BlockReceiver.java:254)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readNextPacket(BlockReceiver.java:341)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:362)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:514)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:356)
> >> > at
> >> >
> >> >
> >>
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:102)
> >> > at java.lang.Thread.run(Thread.java:619)
> >> >
> >> > Larry
> >> >
> >>
> >
> >
> >
>