Awesome, thanks... This is similar to mysql max-conn setting. -Jack
On Sat, Mar 12, 2011 at 11:29 AM, Stack <st...@duboce.net> wrote: > I opened HBASE-3628 to expose the TThreadPoolServer options on the > command-line for thrift server. > St.Ack > > On Sat, Mar 12, 2011 at 11:20 AM, Stack <st...@duboce.net> wrote: > > Via Bryan (and J-D), by default we use the thread pool server from > > Thrift (unless you choose the non-blocking option): > > > > 978 LOG.info("starting HBase ThreadPool Thrift server on " + > > listenAddress + ":" + Integer.toString(listenPort)); > > 979 server = new TThreadPoolServer(processor, serverTransport, > > transportFactory, protocolFactory); > > > > By default, the max threads is: > > > > 63 public int minWorkerThreads = 5; > > 64 public int maxWorkerThreads = Integer.MAX_VALUE; > > > > ... so doing some hacking like below in our ThriftServer.java should > > add in a maximum for your: > > > > diff --git > a/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java > > b/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java > > index 06621ab..74856af 100644 > > --- a/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java > > +++ b/src/main/java/org/apache/hadoop/hbase/thrift/ThriftServer.java > > @@ -69,6 +69,7 @@ import > org.apache.hadoop.hbase.thrift.generated.TRegionInfo; > > import org.apache.hadoop.hbase.thrift.generated.TRowResult; > > import org.apache.hadoop.hbase.util.Bytes; > > import org.apache.thrift.TException; > > +import org.apache.thrift.TProcessorFactory; > > import org.apache.thrift.protocol.TBinaryProtocol; > > import org.apache.thrift.protocol.TCompactProtocol; > > import org.apache.thrift.protocol.TProtocolFactory; > > @@ -911,9 +912,25 @@ public class ThriftServer { > > } else { > > transportFactory = new TTransportFactory(); > > } > > - > > - LOG.info("starting HBase ThreadPool Thrift server on " + > > listenAddress + ":" + Integer.toString(listenPort)); > > - server = new TThreadPoolServer(processor, serverTransport, > > transportFactory, protocolFactory); > > + TThreadPoolServer.Options poolServerOptions = > > + new TThreadPoolServer.Options(); > > + int maxWorkerThreads = Integer.MAX_VALUE; > > + if (cmd.hasOption("maxWorkerThreads")) { > > + try { > > + maxWorkerThreads = > > + Integer.parseInt(cmd.getOptionValue("maxWorkerThreads", > > "" + Integer.MAX_VALUE)); > > + } catch (NumberFormatException e) { > > + LOG.error("Could not parse maxWorkerThreads option", e); > > + printUsageAndExit(options, -1); > > + } > > + } > > + poolServerOptions.maxWorkerThreads = maxWorkerThreads; > > + LOG.info("starting HBase ThreadPool Thrift server on " + > listenAddress + > > + ":" + Integer.toString(listenPort) + > > + ", maxWorkerThreads=" + maxWorkerThreads); > > + server = new TThreadPoolServer(processor, serverTransport, > > + transportFactory, transportFactory, protocolFactory, > protocolFactory, > > + poolServerOptions); > > } > > > > Looks like other useful options to set in there. > > St.Ack > > > > > > On Sat, Mar 12, 2011 at 10:47 AM, Stack <st...@duboce.net> wrote: > >> I don't see any bounding in the thrift code. Asking Bryan.... > >> St.Ack > >> > >> On Sat, Mar 12, 2011 at 10:04 AM, Jack Levin <magn...@gmail.com> wrote: > >>> So our problem is this: when we restart a region server, or it goes > >>> down, hbase slows down, while we send super high frequency thrift > >>> calls from our PHP front-end APP we actually spawn up 20000+ threads on > >>> thrift, and what this > >>> does is destroys all memory on the boxes, and causes DNs just to shut > >>> down, and everything else crash. > >>> > >>> Is there a way to put thread limiter on thrift? Maybe 1000 threads MAX? > >>> > >>> -Jack > >>> > >>> On Sat, Mar 12, 2011 at 3:31 AM, Suraj Varma <svarma...@gmail.com> > wrote: > >>> > >>>> >> to:java.lang.OutOfMemoryError: unable to create new native thread > >>>> > >>>> This indicates that you are oversubscribed on your RAM to the extent > >>>> that the JVM doesn't have any space to create native threads (which > >>>> are allocated outside of the JVM heap.) > >>>> > >>>> You may actually have to _reduce_ your heap sizes to allow more space > >>>> for native threads (do an inventory of all the JVM heaps and don't let > >>>> it go over about 75% of available RAM.) > >>>> Another option is to use the -Xss stack size JVM arg to reduce the per > >>>> thread stack size - set it to 512k or 256k (you may have to > >>>> experiment/perf test a bit to see what's the optimum size. > >>>> Or ... get more RAM ... > >>>> > >>>> --Suraj > >>>> > >>>> On Fri, Mar 11, 2011 at 8:11 PM, Jack Levin <magn...@gmail.com> > wrote: > >>>> > I am noticing following errors also: > >>>> > > >>>> > 2011-03-11 17:52:00,376 ERROR > >>>> > org.apache.hadoop.hdfs.server.datanode.DataNode: > DatanodeRegistration( > >>>> > 10.103.7.3:50010, > storageID=DS-824332190-10.103.7.3-50010-1290043658438, > >>>> > infoPort=50075, ipcPort=50020):DataXceiveServer: Exiting due > >>>> > to:java.lang.OutOfMemoryError: unable to create new native thread > >>>> > at java.lang.Thread.start0(Native Method) > >>>> > at java.lang.Thread.start(Thread.java:597) > >>>> > at > >>>> > > >>>> > org.apache.hadoop.hdfs.server.datanode.DataXceiverServer.run(DataXceiverServer.java:132) > >>>> > at java.lang.Thread.run(Thread.java:619) > >>>> > > >>>> > > >>>> > and this: > >>>> > > >>>> > nf_conntrack: table full, dropping packet. > >>>> > nf_conntrack: table full, dropping packet. > >>>> > nf_conntrack: table full, dropping packet. > >>>> > nf_conntrack: table full, dropping packet. > >>>> > nf_conntrack: table full, dropping packet. > >>>> > nf_conntrack: table full, dropping packet. > >>>> > net_ratelimit: 10 callbacks suppressed > >>>> > nf_conntrack: table full, dropping packet. > >>>> > possible SYN flooding on port 9090. Sending cookies. > >>>> > > >>>> > This seems like a network stack issue? > >>>> > > >>>> > So, does datanode need higher heap than 1GB? Or possible we ran out > of > >>>> RAM > >>>> > for other reasons? > >>>> > > >>>> > -Jack > >>>> > > >>>> > On Thu, Mar 10, 2011 at 1:29 PM, Ryan Rawson <ryano...@gmail.com> > wrote: > >>>> > > >>>> >> Looks like a datanode went down. InterruptedException is how java > >>>> >> uses to interrupt IO in threads, its similar to the EINTR errno. > That > >>>> >> means the actual source of the abort is higher up... > >>>> >> > >>>> >> So back to how InterruptedException works... at some point a thread > in > >>>> >> the JVM decides that the VM should abort. So it calls > >>>> >> thread.interrupt() on all the threads it knows/cares about to > >>>> >> interrupt their IO. That is what you are seeing in the logs. The > root > >>>> >> cause lies above I think. > >>>> >> > >>>> >> Look for the first "Exception" string or any FATAL or ERROR strings > in > >>>> >> the datanode logfiles. > >>>> >> > >>>> >> -ryan > >>>> >> > >>>> >> On Thu, Mar 10, 2011 at 1:03 PM, Jack Levin <magn...@gmail.com> > wrote: > >>>> >> > http://pastebin.com/ZmsyvcVc Here is the regionserver log, they > all > >>>> >> have > >>>> >> > similar stuff, > >>>> >> > > >>>> >> > On Thu, Mar 10, 2011 at 11:34 AM, Stack <st...@duboce.net> > wrote: > >>>> >> > > >>>> >> >> Whats in the regionserver logs? Please put up regionserver and > >>>> >> >> datanode excerpts. > >>>> >> >> Thanks Jack, > >>>> >> >> St.Ack > >>>> >> >> > >>>> >> >> On Thu, Mar 10, 2011 at 10:31 AM, Jack Levin <magn...@gmail.com > > > >>>> wrote: > >>>> >> >> > All was well, until this happen: > >>>> >> >> > > >>>> >> >> > http://pastebin.com/iM1niwrS > >>>> >> >> > > >>>> >> >> > and all regionservers went down, is this xciever issue? > >>>> >> >> > > >>>> >> >> > <property> > >>>> >> >> > <name>dfs.datanode.max.xcievers</name> > >>>> >> >> > <value>12047</value> > >>>> >> >> > </property> > >>>> >> >> > > >>>> >> >> > this is what I have, should I set it higher? > >>>> >> >> > > >>>> >> >> > -Jack > >>>> >> >> > > >>>> >> >> > >>>> >> > > >>>> >> > >>>> > > >>>> > >>> > >> > > >