hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Weihua JIANG <weihua.ji...@gmail.com>
Subject Re: How to speedup Hbase query throughput
Date Thu, 19 May 2011 00:11:19 GMT
All the DNs almost have the same number of blocks. Major compaction
makes no difference.

Thanks
Weihua

2011/5/18 Stack <stack@duboce.net>:
> Are there more blocks on these hot DNs than there are on the cool
> ones?   If you run a major compaction and then run your tests, does it
> make a difference?
> St.Ack
>
> On Tue, May 17, 2011 at 8:03 PM, Weihua JIANG <weihua.jiang@gmail.com> wrote:
>> -ROOT- and .META. table are not served by these hot region servers.
>>
>> I generate the key in random and verified at client by grepping .META.
>> table and record the mapping from each query to its serving region
>> server. It shows that each RS serves almost the same number of query
>> requests.
>>
>> For GC hints, can you give me a link? I only found Todd's posts about
>> GC tuning for write. But, in my case, I only perform query. So, the
>> one I found seems no help to me.
>>
>> Thanks
>> Weihua
>>
>> 2011/5/17 Michael Segel <michael_segel@hotmail.com>:
>>>
>>> Sorry to jump in on the tail end.
>>>
>>> What do you mean to say that they key is generated randomly?
>>>
>>> I mean are you using a key and then applying a SHA-1 hash?
>>>
>>> Which node is serving your -ROOT- and META tables?
>>>
>>> Have you applied the GC hints recommended by Todd L in his blog?
>>>
>>>
>>> Also you said:
>>> '
>>> And almost all these times are spent on DFSClient$BlockReader
>>> to read data from DN.
>>> '
>>> What speed disks are you using and how many disks per node?
>>> (you could be blocked on disk i/o.)
>>>
>>>
>>> -Mike
>>>
>>>
>>> ----------------------------------------
>>>> Date: Tue, 17 May 2011 07:33:34 -0700
>>>> Subject: Re: How to speedup Hbase query throughput
>>>> From: stack@duboce.net
>>>> To: user@hbase.apache.org
>>>>
>>>> Nice analysis.
>>>>
>>>> Can you figure the most popular blocks requested? You could figure
>>>> which files they belong too by grepping the blocks in namenode log.
>>>>
>>>> It is odd that you have the sort of a request profile if your loading
>>>> was even. I'd expect the DN distribution to be even.
>>>>
>>>> Sounds like hdfs-347 would help for sure.
>>>>
>>>> St.Ack
>>>>
>>>>
>>>> On Tue, May 17, 2011 at 6:57 AM, Weihua JIANG  wrote:
>>>> > No. The key is generated randomly. In theory, it shall distributed to
>>>> > all the RSs equally.
>>>> >
>>>> > Thanks
>>>> > Weihua
>>>> >
>>>> > 2011/5/17 Ted Dunning :
>>>> >> Are your keys arranged so that you have a problem with a hot region?
>>>> >>
>>>> >> On Mon, May 16, 2011 at 11:18 PM, Weihua JIANG wrote:
>>>> >>
>>>> >>> I have not applied hdfs-347, but done some other experiments.
>>>> >>>
>>>> >>> I increased client thread to 2000 to put enough pressure on
cluster. I
>>>> >>> disabled RS block cache. The total TPS is still low (with Month+User
>>>> >>> as row key, it is about 1300 for 10 RS+DN and with User+Month
it is
>>>> >>> 700).
>>>> >>>
>>>> >>> I used BTrace to log the time spent on each HTable.get on RS.
It shows
>>>> >>> that most of the GETs use 20~50ms and there are many GETs need
>>>> >>> >1000ms. And almost all these times are spent on DFSClient$BlockReader
>>>> >>> to read data from DN. But, the network usage is not high (<100Mb/s,
we
>>>> >>> have a giganet), so network is not a problem.
>>>> >>>
>>>> >>> Since for each DFS block read, there is a socket connection
created. I
>>>> >>> use netstat to caculate the TCP connections on 50010 port (DN
listen
>>>> >>> port) for each RS+DN server. It shows that there are always
one or two
>>>> >>> DNs have high such connection number (>200) while other DNs
have low
>>>> >>> number (<20). And the high connection DNs have high disk
I/O usage
>>>> >>> (about 100%) while other DNs have low disk I/O.  This phenoma
lasts
>>>> >>> for days and the hot machine is always the hot one.
>>>> >>>
>>>> >>> The high connection number mainly comes from local region server
>>>> >>> request (~80%).
>>>> >>>
>>>> >>> According to the source code of DFSClient, it prefers to use
local DN
>>>> >>> to fetch block. But, why certain machine is so popular? All
my servers
>>>> >>> have almost the same configuration.
>>>> >>>
>>>> >>> 2011/4/29 Stack :
>>>> >>> > Yes, you could try applying hdfs-347 to your hdfs as J-D
suggests.  Do
>>>> >>> > your numbers change if you run your client from more than
one machine?
>>>> >>> > St.Ack
>>>> >>> >
>>>> >>> > On Thu, Apr 28, 2011 at 2:56 PM, Jean-Daniel Cryans
>>>> >>> wrote:
>>>> >>> >> Seems to be a case of HDFS-347.
>>>> >>> >>
>>>> >>> >> J-D
>>>> >>> >>
>>>> >>> >> On Thu, Apr 28, 2011 at 12:55 AM, Weihua JIANG
>>>> >>> wrote:
>>>> >>> >>> After solving HBASE-3561, I successfully run hprof
for RS and DN.
>>>> >>> >>> Since block cache is useless in my case, I disabled
it. I rerun my
>>>> >>> >>> test with 14 RS+DNs and 1 client with 200 threads.
But, the throughput
>>>> >>> >>> is still only about 700. No scalability shown in
this case.
>>>> >>> >>>
>>>> >>> >>> Below is the hot spots in RS:
>>>> >>> >>> CPU SAMPLES BEGIN (total = 1469756) Thu Apr 28
15:43:35 2011
>>>> >>> >>> rank   self  accum   count trace method
>>>> >>> >>>   1 44.33% 44.33%  651504 300612 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   2 19.88% 64.21%  292221 301351 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   3  8.88% 73.09%  130582 300554 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   4  4.43% 77.52%   65106 301248 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   5  4.43% 81.95%   65104 301249 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   6  4.43% 86.38%   65100 301247 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   7  4.43% 90.81%   65061 301266 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   8  4.32% 95.13%   63465 301565 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   9  2.31% 97.43%   33894 301555 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>  10  1.76% 99.19%   25841 301588 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>  11  0.48% 99.67%    7025 301443sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>  12  0.02% 99.69%     341 301568 sun.nio.ch.NativeThread.current
>>>> >>> >>>  13  0.01% 99.71%     187 301535
>>>> >>> >>> org.apache.hadoop.hbase.io.hfile.HFile$Reader.indexSize
>>>> >>> >>>  14  0.01% 99.72%     186 301538
>>>> >>> >>> org.apache.hadoop.hbase.io.hfile.HFile$Reader.indexSize
>>>> >>> >>>  15  0.01% 99.73%     170 301625
>>>> >>> org.apache.hadoop.util.DataChecksum.update
>>>> >>> >>>  16  0.01% 99.74%     164 301579 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>  17  0.01% 99.75%     149 300938 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>
>>>> >>> >>> TRACE 300612:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hbase.ipc.HBaseServer$Listener$Reader.run(HBaseServer.java:305)
>>>> >>> >>>
>>>> >>>  java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
>>>> >>> >>>
>>>> >>>  java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
>>>> >>> >>>        java.lang.Thread.run(Thread.java:619)
>>>> >>> >>> TRACE 301351:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>>> >>> >>>        java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>>> >>> >>>        java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
>>>> >>> >>>        java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>>>> >>> >>>        java.io.DataInputStream.read(DataInputStream.java:132)
>>>> >>> >>>        org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:102)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1389)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:176)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:193)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1243)
>>>> >>> >>> TRACE 300554:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1107)
>>>> >>> >>> TRACE 301248:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hbase.ipc.HBaseServer$Responder.run(HBaseServer.java:588)
>>>> >>> >>> TRACE 301249:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hbase.ipc.HBaseServer$Listener.run(HBaseServer.java:414)
>>>> >>> >>> TRACE 301247:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.mortbay.io.nio.SelectorManager$SelectSet.doSelect(SelectorManager.java:498)
>>>> >>> >>>
>>>> >>>  org.mortbay.io.nio.SelectorManager.doSelect(SelectorManager.java:192)
>>>> >>> >>>
>>>> >>>  org.mortbay.jetty.nio.SelectChannelConnector.accept(SelectChannelConnector.java:124)
>>>> >>> >>>
>>>> >>>  org.mortbay.jetty.AbstractConnector$Acceptor.run(AbstractConnector.java:708)
>>>> >>> >>>
>>>> >>>  org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)
>>>> >>> >>> TRACE 301266:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>>> >>> >>>        java.io.DataInputStream.readFully(DataInputStream.java:178)
>>>> >>> >>>        java.io.DataInputStream.readLong(DataInputStream.java:399)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:120)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2638)
>>>> >>> >>> TRACE 301565:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>>> >>> >>>        java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>>> >>> >>>        java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
>>>> >>> >>>        java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>>>> >>> >>>        java.io.DataInputStream.read(DataInputStream.java:132)
>>>> >>> >>>        org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:102)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.DFSClient$BlockReader.readChunk(DFSClient.java:1389)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:237)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:189)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:158)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.DFSClient$BlockReader.read(DFSClient.java:1249)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.fs.FSInputChecker.readFully(FSInputChecker.java:384)
>>>> >>> >>>
>>>> >>> >>>
>>>> >>> >>> And the hot spot of DN is:
>>>> >>> >>> CPU SAMPLES BEGIN (total = 1123925) Thu Apr 28
15:43:49 2011
>>>> >>> >>> rank   self  accum   count trace method
>>>> >>> >>>   1 12.38% 12.38%  139155 301547 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   2 12.38% 24.76%  139154 301548 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   3 10.76% 35.53%  120969 300368 java.net.PlainSocketImpl.socketAccept
>>>> >>> >>>   4 10.74% 46.27%  120753 301213 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   5 10.74% 57.01%  120744 301381 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   6 10.74% 67.76%  120744 301384 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   7 10.74% 78.50%  120743 301386 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>   8 10.68% 89.18%  120019 301383
>>>> >>> sun.nio.ch.ServerSocketChannelImpl.accept0
>>>> >>> >>>   9  8.11% 97.29%   91174 301475 sun.nio.ch.FileChannelImpl.transferTo0
>>>> >>> >>>  10  1.44% 98.73%   16172 301479 java.io.FileInputStream.readBytes
>>>> >>> >>>  11  0.40% 99.13%    4496 301477 java.io.FileInputStream.readBytes
>>>> >>> >>>  12  0.23% 99.35%    2531 301497 sun.nio.ch.EPollArrayWrapper.epollWait
>>>> >>> >>>  13  0.17% 99.53%    1958 301461 java.io.FileInputStream.readBytes
>>>> >>> >>>  14  0.04% 99.57%     458 301468 java.io.UnixFileSystem.delete0
>>>> >>> >>>  15  0.03% 99.59%     289 301520 java.lang.Thread.start0
>>>> >>> >>>  16  0.02% 99.62%     278 301471
>>>> >>> org.apache.hadoop.util.DataChecksum.update
>>>> >>> >>>  17  0.02% 99.64%     204 301562
>>>> >>> java.io.UnixFileSystem.getBooleanAttributes0
>>>> >>> >>>  18  0.01% 99.65%     117 301587 sun.nio.ch.EPollArrayWrapper.epollCtl
>>>> >>> >>>  19  0.01% 99.66%     113 301493 sun.nio.ch.FileDispatcher.write0
>>>> >>> >>>
>>>> >>> >>> TRACE 301547:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>>> >>> >>>        java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>>> >>> >>>        java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
>>>> >>> >>>        java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>>>> >>> >>>        java.io.DataInputStream.read(DataInputStream.java:132)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readToBuf(BlockReceiver.java:267)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockReceiver.readNextPacket(BlockReceiver.java:314)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:378)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:534)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:417)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:122)
>>>> >>> >>> TRACE 301548:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>>>> >>> >>>        java.io.DataInputStream.readFully(DataInputStream.java:178)
>>>> >>> >>>        java.io.DataInputStream.readLong(DataInputStream.java:399)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:120)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:894)
>>>> >>> >>>        java.lang.Thread.run(Thread.java:619)
>>>> >>> >>> TRACE 300368:
>>>> >>> >>>
>>>> >>>  java.net.PlainSocketImpl.socketAccept(PlainSocketImpl.java:Unknown
line)
>>>> >>> >>>        java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
>>>> >>> >>>        java.net.ServerSocket.implAccept(ServerSocket.java:453)
>>>> >>> >>>        java.net.ServerSocket.accept(ServerSocket.java:421)
>>>> >>> >>>
>>>> >>>  sun.management.jmxremote.LocalRMIServerSocketFactory$1.accept(LocalRMIServerSocketFactory.java:34)
>>>> >>> >>>
>>>> >>>  sun.rmi.transport.tcp.TCPTransport$AcceptLoop.executeAcceptLoop(TCPTransport.java:369)
>>>> >>> >>>
>>>> >>>  sun.rmi.transport.tcp.TCPTransport$AcceptLoop.run(TCPTransport.java:341)
>>>> >>> >>>        java.lang.Thread.run(Thread.java:619)
>>>> >>> >>> TRACE 301213:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>
>>>> >>>  org.mortbay.io.nio.SelectorManager$SelectSet.doSelect(SelectorManager.java:498)
>>>> >>> >>>
>>>> >>>  org.mortbay.io.nio.SelectorManager.doSelect(SelectorManager.java:192)
>>>> >>> >>>
>>>> >>>  org.mortbay.jetty.nio.SelectChannelConnector.accept(SelectChannelConnector.java:124)
>>>> >>> >>>
>>>> >>>  org.mortbay.jetty.AbstractConnector$Acceptor.run(AbstractConnector.java:708)
>>>> >>> >>>
>>>> >>>  org.mortbay.thread.QueuedThreadPool$PoolThread.run(QueuedThreadPool.java:582)
>>>> >>> >>> TRACE 301381:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:352)
>>>> >>> >>> TRACE 301384:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>        org.apache.hadoop.ipc.Server$Responder.run(Server.java:636)
>>>> >>> >>> TRACE 301386:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.EPollArrayWrapper.epollWait(EPollArrayWrapper.java:Unknown
line)
>>>> >>> >>>        sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215)
>>>> >>> >>>        sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
>>>> >>> >>>        sun.nio.ch.SelectorImpl.select(SelectorImpl.java:84)
>>>> >>> >>>        org.apache.hadoop.ipc.Server$Listener.run(Server.java:468)
>>>> >>> >>> TRACE 301383:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.ServerSocketChannelImpl.accept0(ServerSocketChannelImpl.java:Unknown
>>>> >>> >>> line)
>>>> >>> >>>
>>>> >>>  sun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:145)
>>>> >>> >>>
>>>> >>>  sun.nio.ch.ServerSocketAdaptor.accept(ServerSocketAdaptor.java:84)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiverServer.run(DataXceiverServer.java:130)
>>>> >>> >>>        java.lang.Thread.run(Thread.java:619)
>>>> >>> >>> TRACE 301475:
>>>> >>> >>>
>>>> >>>  sun.nio.ch.FileChannelImpl.transferTo0(FileChannelImpl.java:Unknown
line)
>>>> >>> >>>
>>>> >>>  sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:415)
>>>> >>> >>>        sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:516)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:199)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:350)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:214)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:114)
>>>> >>> >>> TRACE 301479:
>>>> >>> >>>        java.io.FileInputStream.readBytes(FileInputStream.java:Unknown
>>>> >>> line)
>>>> >>> >>>        java.io.FileInputStream.read(FileInputStream.java:199)
>>>> >>> >>>        java.io.FilterInputStream.read(FilterInputStream.java:116)
>>>> >>> >>>        java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
>>>> >>> >>>        java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
>>>> >>> >>>        java.io.BufferedInputStream.read(BufferedInputStream.java:317)
>>>> >>> >>>        java.io.DataInputStream.readFully(DataInputStream.java:178)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:271)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:436)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXceiver.java:214)
>>>> >>> >>>
>>>> >>>  org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:114)
>>>> >>> >>>
>>>> >>> >>> Thanks
>>>> >>> >>> Weihua
>>>> >>> >>>
>>>> >>> >>> 2011/4/28 Weihua JIANG :
>>>> >>> >>>> I am using CDH3U0. It is HBase 0.90.1, I think.
>>>> >>> >>>>
>>>> >>> >>>> Thanks
>>>> >>> >>>> Weihua
>>>> >>> >>>>
>>>> >>> >>>> 2011/4/28 Stack :
>>>> >>> >>>>> On Tue, Apr 26, 2011 at 6:02 PM, Weihua
JIANG <
>>>> >>> weihua.jiang@gmail.com> wrote:
>>>> >>> >>>>>> I tried to enable HPROF on RS, but
failed. If I added the HPROF
>>>> >>> agent
>>>> >>> >>>>>> in hbase-env.sh, RS startup reports
an error said HPROF can't be
>>>> >>> >>>>>> loaded twice. But, I am sure I only
enabled it once. I don't know
>>>> >>> >>>>>> where the problem is.
>>>> >>> >>>>>>
>>>> >>> >>>>>
>>>> >>> >>>>> This sounds like 'HBASE-3561  OPTS arguments
are duplicated'  Are you
>>>> >>> >>>>> running 0.90.2?
>>>> >>> >>>>>
>>>> >>> >>>>> St.Ack
>>>> >>> >>>>>
>>>> >>> >>>>
>>>> >>> >>>
>>>> >>> >>
>>>> >>> >
>>>> >>>
>>>> >>
>>>> >
>>>
>>
>

Mime
View raw message