Re: Spark SQL concurrent runs fails with java.util.concurrent.TimeoutException: Futures timed out after [300 seconds]

2016-08-19 Thread Davies Liu
The query failed to finish broadcast in 5 minutes, you could decrease
the broadcast threshold (spark.sql.autoBroadcastJoinThreshold) or
increase the conf: spark.sql.broadcastTimeout

On Tue, Jun 28, 2016 at 3:35 PM, Jesse F Chen  wrote:
>
> With the Spark 2.0 build from 0615, when running 4-user concurrent SQL tests 
> against Spark SQL on 1TB TPCDS, we are seeing
> consistently the following exceptions:
>
> 10:35:33 AM: 16/06/27 23:40:37 INFO scheduler.TaskSetManager: Finished task 
> 412.0 in stage 819.0 (TID 270396) in 8468 ms on 9.30.148.101 (417/581)
> 16/06/27 23:40:37 ERROR thriftserver.SparkExecuteStatementOperation: Error 
> executing query, currentState RUNNING,
> org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
> Exchange SinglePartition
> +- *HashAggregate(key=[], 
> functions=[partial_sum(cs_ext_discount_amt#100849)], output=[sum#101124])
> +- *Project [cs_ext_discount_amt#100849]
>
> 
> at 
> org.apache.spark.sql.execution.exchange.ShuffleExchange$$anonfun$doExecute$1.apply(ShuffleExchange.scala:113)
> at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:49)
> ... 40 more
> Caused by: java.util.concurrent.TimeoutException: Futures timed out after 
> [300 seconds]
> at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:219)
> at scala.concurrent.impl.Promise$DefaultPromise.result(Promise.scala:223)
> at scala.concurrent.Await$$anonfun$result$1.apply(package.scala:190)
> at 
> scala.concurrent.BlockContext$DefaultBlockContext$.blockOn(BlockContext.scala:53)
> at scala.concurrent.Await$.result(package.scala:190)
> at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:190)
>
>
> The longest query would complete in about 700 seconds, and I think we need to 
> increase the futures timeout value. However,
> I tried the 'spark.network.timeout' setting to 700 via the '--conf' facility 
> but it does not seem to control this particular timeout value.
> In other words, it stays at "300 seconds" no matter what value I give it. I 
> also played with the spark.rpc.askTimeout setting which
> does not affect this 300-second value.
>
> Could someone tell me which parameter I need to change in order to control it?
>
>
> JESSE CHEN
> Big Data Performance | IBM Analytics
>
> Office: 408 463 2296
> Mobile: 408 828 9068
> Email: jfc...@us.ibm.com
>
>

-
To unsubscribe e-mail: user-unsubscr...@spark.apache.org



Spark SQL concurrent runs fails with java.util.concurrent.TimeoutException: Futures timed out after [300 seconds]

2016-06-28 Thread Jesse F Chen

With the Spark 2.0 build from 0615, when running 4-user concurrent SQL
tests against Spark SQL on 1TB TPCDS, we are seeing
consistently the following exceptions:

10:35:33 AM: 16/06/27 23:40:37 INFO scheduler.TaskSetManager: Finished task
412.0 in stage 819.0 (TID 270396) in 8468 ms on 9.30.148.101 (417/581)
16/06/27 23:40:37 ERROR thriftserver.SparkExecuteStatementOperation: Error
executing query, currentState RUNNING,
org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute,
tree:
Exchange SinglePartition
+- *HashAggregate(key=[], functions=[partial_sum
(cs_ext_discount_amt#100849)], output=[sum#101124])
   +- *Project [cs_ext_discount_amt#100849]


at org.apache.spark.sql.execution.exchange.ShuffleExchange$$anonfun
$doExecute$1.apply(ShuffleExchange.scala:113)
at org.apache.spark.sql.catalyst.errors.package$.attachTree
(package.scala:49)
... 40 more
Caused by: java.util.concurrent.TimeoutException: Futures timed out after
[300 seconds]
at scala.concurrent.impl.Promise$DefaultPromise.ready
(Promise.scala:219)
at scala.concurrent.impl.Promise$DefaultPromise.result
(Promise.scala:223)
at scala.concurrent.Await$$anonfun$result$1.apply
(package.scala:190)
at scala.concurrent.BlockContext$DefaultBlockContext$.blockOn
(BlockContext.scala:53)
at scala.concurrent.Await$.result(package.scala:190)
at org.apache.spark.util.ThreadUtils$.awaitResult
(ThreadUtils.scala:190)


The longest query would complete in about 700 seconds, and I think we need
to increase the futures timeout value. However,
I tried the 'spark.network.timeout' setting to 700 via the '--conf'
facility but it does not seem to control this particular timeout value.
In other words, it stays at "300 seconds" no matter what value I give it. I
also played with the spark.rpc.askTimeout setting which
does not affect this 300-second value.

Could someone tell me which parameter I need to change in order to control
it?


  

  

  

  

  

  
   JESSE CHEN   
  
   Big Data Performance | IBM Analytics 
  

  
   Office:  408 463 2296
  
   Mobile: 408 828 9068 
  
   Email:   jfc...@us.ibm.com