[ https://issues.apache.org/jira/browse/SPARK-18745?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15735626#comment-15735626 ]
Apache Spark commented on SPARK-18745: -------------------------------------- User 'kiszk' has created a pull request for this issue: https://github.com/apache/spark/pull/16235 > java.lang.IndexOutOfBoundsException running query 68 Spark SQL on (100TB) > ------------------------------------------------------------------------- > > Key: SPARK-18745 > URL: https://issues.apache.org/jira/browse/SPARK-18745 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.1.0, 2.2.0 > Reporter: JESSE CHEN > Assignee: Kazuaki Ishizaki > Priority: Critical > > Running query 68 with decreased executor memory (using 12GB executors instead > of 24GB) on 100TB parquet database using the Spark master dated 11/04 gave > IndexOutOfBoundsException. > The query is as follows: > {noformat} > [select c_last_name > ,c_first_name > ,ca_city > ,bought_city > ,ss_ticket_number > ,extended_price > ,extended_tax > ,list_price > from (select ss_ticket_number > ,ss_customer_sk > ,ca_city bought_city > ,sum(ss_ext_sales_price) extended_price > ,sum(ss_ext_list_price) list_price > ,sum(ss_ext_tax) extended_tax > from store_sales > ,date_dim > ,store > ,household_demographics > ,customer_address > where store_sales.ss_sold_date_sk = date_dim.d_date_sk > and store_sales.ss_store_sk = store.s_store_sk > and store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk > and store_sales.ss_addr_sk = customer_address.ca_address_sk > and date_dim.d_dom between 1 and 2 > and (household_demographics.hd_dep_count = 8 or > household_demographics.hd_vehicle_count= -1) > and date_dim.d_year in (2000,2000+1,2000+2) > and store.s_city in ('Plainview','Rogers') > group by ss_ticket_number > ,ss_customer_sk > ,ss_addr_sk,ca_city) dn > ,customer > ,customer_address current_addr > where ss_customer_sk = c_customer_sk > and customer.c_current_addr_sk = current_addr.ca_address_sk > and current_addr.ca_city <> bought_city > order by c_last_name > ,ss_ticket_number > limit 100] > {noformat} > Spark output that showed the exception: > {noformat} > org.apache.spark.SparkException: Exception thrown in awaitResult: > at > org.apache.spark.util.ThreadUtils$.awaitResultInForkJoinSafely(ThreadUtils.scala:215) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:131) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.SparkPlan.executeBroadcast(SparkPlan.scala:123) > at > org.apache.spark.sql.execution.exchange.ReusedExchangeExec.doExecuteBroadcast(Exchange.scala:61) > at > org.apache.spark.sql.execution.InputAdapter.doExecuteBroadcast(WholeStageCodegenExec.scala:231) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.SparkPlan.executeBroadcast(SparkPlan.scala:123) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.prepareBroadcast(BroadcastHashJoinExec.scala:98) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:197) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) > at > org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) > at > org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) > at > org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.consume(SortMergeJoinExec.scala:35) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.doProduce(SortMergeJoinExec.scala:560) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.produce(SortMergeJoinExec.scala:35) > at > org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) > at > org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:313) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:354) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) > at > org.apache.spark.sql.execution.TakeOrderedAndProjectExec.executeCollect(limit.scala:133) > at > org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:295) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:134) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:133) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) > at > org.apache.spark.sql.execution.QueryExecution.hiveResultString(QueryExecution.scala:133) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:331) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:311) > at > org.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:409) > at org.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:425) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:173) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) > 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:498) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:738) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:187) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:212) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:126) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.IndexOutOfBoundsException > at java.io.ObjectOutputStream.write(ObjectOutputStream.java:707) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap$$anonfun$writeExternal$4.apply(HashedRelation.scala:698) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap$$anonfun$writeExternal$4.apply(HashedRelation.scala:698) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.writeLongArray(HashedRelation.scala:675) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.write(HashedRelation.scala:694) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.writeExternal(HashedRelation.scala:698) > at > java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > org.apache.spark.sql.execution.joins.LongHashedRelation.writeExternal(HashedRelation.scala:788) > at > java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:43) > at > org.apache.spark.broadcast.TorrentBroadcast$$anonfun$blockifyObject$2.apply(TorrentBroadcast.scala:236) > at > org.apache.spark.broadcast.TorrentBroadcast$$anonfun$blockifyObject$2.apply(TorrentBroadcast.scala:236) > at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1302) > at > org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:237) > at > org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:107) > at > org.apache.spark.broadcast.TorrentBroadcast.<init>(TorrentBroadcast.scala:86) > at > org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34) > at > org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:56) > at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1413) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:97) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:75) > at > org.apache.spark.sql.execution.SQLExecution$.withExecutionId(SQLExecution.scala:94) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:74) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:74) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24) > 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) > org.apache.spark.SparkException: Exception thrown in awaitResult: > at > org.apache.spark.util.ThreadUtils$.awaitResultInForkJoinSafely(ThreadUtils.scala:215) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:131) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.SparkPlan.executeBroadcast(SparkPlan.scala:123) > at > org.apache.spark.sql.execution.exchange.ReusedExchangeExec.doExecuteBroadcast(Exchange.scala:61) > at > org.apache.spark.sql.execution.InputAdapter.doExecuteBroadcast(WholeStageCodegenExec.scala:231) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:124) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.SparkPlan.executeBroadcast(SparkPlan.scala:123) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.prepareBroadcast(BroadcastHashJoinExec.scala:98) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.codegenInner(BroadcastHashJoinExec.scala:197) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doConsume(BroadcastHashJoinExec.scala:82) > at > org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) > at > org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:68) > at > org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:153) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.consume(SortMergeJoinExec.scala:35) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.doProduce(SortMergeJoinExec.scala:560) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.joins.SortMergeJoinExec.produce(SortMergeJoinExec.scala:35) > at > org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.doProduce(BroadcastHashJoinExec.scala:77) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.produce(BroadcastHashJoinExec.scala:38) > at > org.apache.spark.sql.execution.ProjectExec.doProduce(basicPhysicalOperators.scala:46) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:83) > at > org.apache.spark.sql.execution.CodegenSupport$$anonfun$produce$1.apply(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at > org.apache.spark.sql.execution.CodegenSupport$class.produce(WholeStageCodegenExec.scala:78) > at > org.apache.spark.sql.execution.ProjectExec.produce(basicPhysicalOperators.scala:36) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doCodeGen(WholeStageCodegenExec.scala:313) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:354) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:135) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:132) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:113) > at > org.apache.spark.sql.execution.TakeOrderedAndProjectExec.executeCollect(limit.scala:133) > at > org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:295) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:134) > at > org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$4.apply(QueryExecution.scala:133) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) > at > org.apache.spark.sql.execution.QueryExecution.hiveResultString(QueryExecution.scala:133) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:331) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376) > at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:311) > at > org.apache.hadoop.hive.cli.CliDriver.processReader(CliDriver.java:409) > at org.apache.hadoop.hive.cli.CliDriver.processFile(CliDriver.java:425) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:173) > at > org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) > 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:498) > at > org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:738) > at > org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:187) > at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:212) > at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:126) > at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) > Caused by: java.lang.IndexOutOfBoundsException > at java.io.ObjectOutputStream.write(ObjectOutputStream.java:707) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap$$anonfun$writeExternal$4.apply(HashedRelation.scala:698) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap$$anonfun$writeExternal$4.apply(HashedRelation.scala:698) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.writeLongArray(HashedRelation.scala:675) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.write(HashedRelation.scala:694) > at > org.apache.spark.sql.execution.joins.LongToUnsafeRowMap.writeExternal(HashedRelation.scala:698) > at > java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > org.apache.spark.sql.execution.joins.LongHashedRelation.writeExternal(HashedRelation.scala:788) > at > java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459) > at > java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430) > at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178) > at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348) > at > org.apache.spark.serializer.JavaSerializationStream.writeObject(JavaSerializer.scala:43) > at > org.apache.spark.broadcast.TorrentBroadcast$$anonfun$blockifyObject$2.apply(TorrentBroadcast.scala:236) > at > org.apache.spark.broadcast.TorrentBroadcast$$anonfun$blockifyObject$2.apply(TorrentBroadcast.scala:236) > at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1302) > at > org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:237) > at > org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:107) > at > org.apache.spark.broadcast.TorrentBroadcast.<init>(TorrentBroadcast.scala:86) > at > org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34) > at > org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:56) > at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1413) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:97) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1$$anonfun$apply$1.apply(BroadcastExchangeExec.scala:75) > at > org.apache.spark.sql.execution.SQLExecution$.withExecutionId(SQLExecution.scala:94) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:74) > at > org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anonfun$relationFuture$1.apply(BroadcastExchangeExec.scala:74) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24) > at > scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24) > 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) > {noformat} > Likely an integer overflow 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