You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Gengliang Wang (Jira)" <ji...@apache.org> on 2019/09/17 15:45:00 UTC
[jira] [Created] (SPARK-29122) Propagate all the SQL conf to
executors in SQLQueryTestSuite
Gengliang Wang created SPARK-29122:
--------------------------------------
Summary: Propagate all the SQL conf to executors in SQLQueryTestSuite
Key: SPARK-29122
URL: https://issues.apache.org/jira/browse/SPARK-29122
Project: Spark
Issue Type: Improvement
Components: SQL, Tests
Affects Versions: 3.0.0
Reporter: Gengliang Wang
Currently, the execution of SQLQueryTestSuite execute the SQL query as following:
{code:java}
val answer = hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg)
{code}
We should propagate all the SQL conf to executors:
{code:java}
val answer =
SQLExecution.withNewExecutionId(session, df.queryExecution, Some(sql)) {
hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg)
}
{code}
However, there is exception after changes when running "pgSQL/numeric.sql":
{code:java}
CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet;
....
select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4): QueryOutput(select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4),struct<>,java.lang.IllegalArgumentException
[info] requirement failed: MutableProjection cannot use UnsafeRow for output data types: decimal(38,0)) (SQLQueryTestSuite.scala:380)
{code}
To reproduce, run with sbt::
{code:java}
testOnly org.apache.spark.sql.SQLQueryTestSuite -- -z "pgSQL/numeric.sql"
{code}
The back trace:
{code:java}
at scala.Predef$.require(Predef.scala:281)
at org.apache.spark.sql.catalyst.expressions.InterpretedMutableProjection.target(InterpretedMutableProjection.scala:56)
at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.createNewAggregationBuffer(TungstenAggregationIterator.scala:129)
at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.<init>(TungstenAggregationIterator.scala:156)
at org.apache.spark.sql.execution.aggregate.HashAggregateExec.$anonfun$doExecute$2(HashAggregateExec.scala:136)
at org.apache.spark.sql.execution.aggregate.HashAggregateExec.$anonfun$doExecute$2$adapted(HashAggregateExec.scala:111)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2(RDD.scala:867)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndex$2$adapted(RDD.scala:867)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:127)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:449)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:452)
at java.util.concurrent.ThreadPoolExecuDD.compute(MapPartitionsRDD.scala:52)tor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:748)
{code}
I tried the queries with spark-sql, it doesn't reproduce. I think it is the problem of the test suite.
--
This message was sent by Atlassian Jira
(v8.3.2#803003)
---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org