[ https://issues.apache.org/jira/browse/SPARK-13935?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15197491#comment-15197491 ]
Tao Wang edited comment on SPARK-13935 at 3/16/16 3:20 PM: ----------------------------------------------------------- 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=0x00007fc390c4c800 nid=0x5fcb waiting for monitor entry [0x00007fc367ac3000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:262) - {color: red}waiting to lock <0x0000000702871ce8> (a org.apache.spark.sql.hive.client.IsolatedClientLoader){color} 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.<init>(SQLContext.scala:540) at org.apache.spark.sql.hive.InnerHiveContext.<init>(InnerHiveContext.scala:102) at org.apache.spark.sql.hive.HiveContext.<init>(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=0x00007fc3918ad800 nid=0x954a runnable [0x00007fc3722da000] 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 <0x00000007141c4f38> (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) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_partition(ThriftHiveMetastore.java:2347) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_partition(ThriftHiveMetastore.java:2332) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.alter_partition(HiveMetaStoreClient.java:1427) at sun.reflect.GeneratedMethodAccessor97.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156) at com.sun.proxy.$Proxy23.alter_partition(Unknown Source) at sun.reflect.GeneratedMethodAccessor97.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2107) - locked <0x00000007141c4dd8> (a org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler) at com.sun.proxy.$Proxy23.alter_partition(Unknown Source) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartition(Hive.java:574) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartition(Hive.java:550) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartitionSpec(Hive.java:1927) at org.apache.hadoop.hive.ql.metadata.Hive.getPartition(Hive.java:1887) at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1419) at org.apache.hadoop.hive.ql.metadata.Hive.loadDynamicPartitions(Hive.java:1605) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.sql.hive.client.Shim_v1_2.loadDynamicPartitions(HiveShim.scala:590) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply$mcV$sp(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:316) at org.apache.spark.sql.hive.client.ClientWrapper.liftedTree1$1(ClientWrapper.scala:268) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:267) -{color: red} locked <0x0000000702871ce8> (a org.apache.spark.sql.hive.client.IsolatedClientLoader) {color} at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:305) at org.apache.spark.sql.hive.client.ClientWrapper.loadDynamicPartitions(ClientWrapper.scala:972) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult$lzycompute(InsertIntoHiveTable.scala:239) - locked <0x000000070291a3f8> (a org.apache.spark.sql.hive.InnerHiveContext$$anon$3) - locked <0x000000071435cdc8> (a org.apache.spark.sql.hive.execution.InsertIntoHiveTable) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult(InsertIntoHiveTable.scala:124) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.doExecute(InsertIntoHiveTable.scala:356) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:145) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:143) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:143) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:1239) - locked <0x000000071435cfc0> (a org.apache.spark.sql.hive.InnerHiveContext$QueryExecution) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:1239) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:1040) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:226) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:161) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:158) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1711) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:171) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) 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) {quote} was (Author: wangtao): 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=0x00007fc390c4c800 nid=0x5fcb waiting for monitor entry [0x00007fc367ac3000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:262) - waiting to lock <0x0000000702871ce8> (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.<init>(SQLContext.scala:540) at org.apache.spark.sql.hive.InnerHiveContext.<init>(InnerHiveContext.scala:102) at org.apache.spark.sql.hive.HiveContext.<init>(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=0x00007fc3918ad800 nid=0x954a runnable [0x00007fc3722da000] 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 <0x00000007141c4f38> (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) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_alter_partition(ThriftHiveMetastore.java:2347) at org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.alter_partition(ThriftHiveMetastore.java:2332) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.alter_partition(HiveMetaStoreClient.java:1427) at sun.reflect.GeneratedMethodAccessor97.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:156) at com.sun.proxy.$Proxy23.alter_partition(Unknown Source) at sun.reflect.GeneratedMethodAccessor97.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler.invoke(HiveMetaStoreClient.java:2107) - locked <0x00000007141c4dd8> (a org.apache.hadoop.hive.metastore.HiveMetaStoreClient$SynchronizedHandler) at com.sun.proxy.$Proxy23.alter_partition(Unknown Source) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartition(Hive.java:574) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartition(Hive.java:550) at org.apache.hadoop.hive.ql.metadata.Hive.alterPartitionSpec(Hive.java:1927) at org.apache.hadoop.hive.ql.metadata.Hive.getPartition(Hive.java:1887) at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1419) at org.apache.hadoop.hive.ql.metadata.Hive.loadDynamicPartitions(Hive.java:1605) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at org.apache.spark.sql.hive.client.Shim_v1_2.loadDynamicPartitions(HiveShim.scala:590) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply$mcV$sp(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$loadDynamicPartitions$1.apply(ClientWrapper.scala:973) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:316) at org.apache.spark.sql.hive.client.ClientWrapper.liftedTree1$1(ClientWrapper.scala:268) at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:267) - locked <0x0000000702871ce8> (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.loadDynamicPartitions(ClientWrapper.scala:972) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult$lzycompute(InsertIntoHiveTable.scala:239) - locked <0x000000070291a3f8> (a org.apache.spark.sql.hive.InnerHiveContext$$anon$3) - locked <0x000000071435cdc8> (a org.apache.spark.sql.hive.execution.InsertIntoHiveTable) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.sideEffectResult(InsertIntoHiveTable.scala:124) at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.doExecute(InsertIntoHiveTable.scala:356) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:145) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:143) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:143) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:1239) - locked <0x000000071435cfc0> (a org.apache.spark.sql.hive.InnerHiveContext$QueryExecution) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:1239) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144) at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:1040) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:226) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:161) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:158) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1711) at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:171) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) 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) {quote} > 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 > Reporter: Tao Wang > Priority: Critical > > We run a sql like "insert overwrite table store_returns partition > (sr_returned_date) select xxxxxx" 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