[jira] [Commented] (SPARK-13935) Other clients' connection hang up when someone do huge load
[ https://issues.apache.org/jira/browse/SPARK-13935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15197491#comment-15197491 ] Tao Wang commented on SPARK-13935: -- after checking the latest codes on github, i believe same problem should be with it too. below is the segment of jstack information. {quote} "HiveServer2-Handler-Pool: Thread-220" #220 daemon prio=5 os_prio=0 tid=0x7fc390c4c800 nid=0x5fcb waiting for monitor entry [0x7fc367ac3000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:262) - waiting to lock <0x000702871ce8> (a org.apache.spark.sql.hive.client.IsolatedClientLoader) at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:305) at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:885) at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:822) at org.apache.spark.sql.hive.InnerHiveContext.setConf(InnerHiveContext.scala:618) at org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:541) at org.apache.spark.sql.SQLContext$$anonfun$6.apply(SQLContext.scala:540) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at scala.collection.AbstractIterator.foreach(Iterator.scala:1157) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at org.apache.spark.sql.SQLContext.(SQLContext.scala:540) at org.apache.spark.sql.hive.InnerHiveContext.(InnerHiveContext.scala:102) at org.apache.spark.sql.hive.HiveContext.(HiveContext.scala:55) at org.apache.spark.sql.hive.HiveContext.newSession(HiveContext.scala:80) at org.apache.spark.sql.hive.thriftserver.SparkSQLSessionManager.openSession(SparkSQLSessionManager.scala:78) at org.apache.hive.service.cli.CLIService.openSessionWithImpersonation(CLIService.java:189) at org.apache.hive.service.cli.thrift.ThriftCLIService.getSessionHandle(ThriftCLIService.java:654) at org.apache.hive.service.cli.thrift.ThriftCLIService.OpenSession(ThriftCLIService.java:522) at org.apache.hive.service.cli.thrift.TCLIService$Processor$OpenSession.getResult(TCLIService.java:1257) at org.apache.hive.service.cli.thrift.TCLIService$Processor$OpenSession.getResult(TCLIService.java:1242) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge$Server$TUGIAssumingProcessor.process(HadoopThriftAuthBridge.java:690) at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:285) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ... "pool-27-thread-18" #299 prio=5 os_prio=0 tid=0x7fc3918ad800 nid=0x954a runnable [0x7fc3722da000] java.lang.Thread.State: RUNNABLE at java.net.SocketInputStream.socketRead0(Native Method) at java.net.SocketInputStream.socketRead(SocketInputStream.java:116) at java.net.SocketInputStream.read(SocketInputStream.java:170) at java.net.SocketInputStream.read(SocketInputStream.java:141) at java.io.BufferedInputStream.fill(BufferedInputStream.java:246) at java.io.BufferedInputStream.read1(BufferedInputStream.java:286) at java.io.BufferedInputStream.read(BufferedInputStream.java:345) - locked <0x0007141c4f38> (a java.io.BufferedInputStream) at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127) at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:376) at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:453) at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:435) at org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:37) at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) at org.apache.hadoop.hive.thrift.TFilterTransport.readAll(TFilterTransport.java:62) at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:429) at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:318) at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:219) at org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:69)
[jira] [Commented] (SPARK-13935) Other clients' connection hang up when someone do huge load
[ https://issues.apache.org/jira/browse/SPARK-13935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15197554#comment-15197554 ] Tao Wang commented on SPARK-13935: -- [~marmbrus][~liancheng][~chenghao] > Other clients' connection hang up when someone do huge load > --- > > Key: SPARK-13935 > URL: https://issues.apache.org/jira/browse/SPARK-13935 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.5.0, 1.5.2, 1.6.0, 1.6.1 >Reporter: Tao Wang >Priority: Critical > > We run a sql like "insert overwrite table store_returns partition > (sr_returned_date) select xx" using beeline then it will block other > beeline connection while invoke the Hive method via > "ClientWrapper.loadDynamicPartitions". > The reason is that "withHiveState" will lock "clientLoader". Sadly when a new > client comes, it will invoke "setConf" methods which is also sychronized with > "clientLoader". > So the problem is that if the first sql took very long time to run, then all > other client could not connect to thrift server successfully. > We tested on release 1.5.1. not sure if latest release has same issue. -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org