[ https://issues.apache.org/jira/browse/SPARK-8623?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14601644#comment-14601644 ]
Sandy Ryza commented on SPARK-8623: ----------------------------------- I took a look at the line numbers and it seems like this can occur when a null Configuration object is passed in to the constructor of a Hadoop Configuration: https://github.com/apache/hadoop/blob/branch-2.6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java#L693 It's puzzling to me how this could occur. The configuration that's passed in comes from a SerializableConfiguration, but the first line of SerializableConfiguration's readObject method instantiates a Configuration object, so not sure how that could end up null. I'm pretty sure the broadcasted conf is always non-null as well. > Some queries in spark-sql lead to NullPointerException when using Yarn > ---------------------------------------------------------------------- > > Key: SPARK-8623 > URL: https://issues.apache.org/jira/browse/SPARK-8623 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 1.5.0 > Environment: Hadoop 2.6, Kerberos > Reporter: Bolke de Bruin > > The following query was executed using "spark-sql --master yarn-client" on > 1.5.0-SNAPSHOT: > select * from wcs.geolite_city limit 10; > This lead to the following error: > 15/06/25 09:38:37 WARN scheduler.TaskSetManager: Lost task 0.0 in stage 0.0 > (TID 0, lxhnl008.ad.ing.net): java.lang.NullPointerException > at org.apache.hadoop.conf.Configuration.<init>(Configuration.java:693) > at org.apache.hadoop.mapred.JobConf.<init>(JobConf.java:442) > at org.apache.hadoop.mapreduce.Job.<init>(Job.java:131) > at > org.apache.spark.sql.sources.SqlNewHadoopRDD.getJob(SqlNewHadoopRDD.scala:83) > at > org.apache.spark.sql.sources.SqlNewHadoopRDD.getConf(SqlNewHadoopRDD.scala:89) > at > org.apache.spark.sql.sources.SqlNewHadoopRDD$$anon$1.<init>(SqlNewHadoopRDD.scala:127) > at > org.apache.spark.sql.sources.SqlNewHadoopRDD.compute(SqlNewHadoopRDD.scala:124) > at > org.apache.spark.sql.sources.SqlNewHadoopRDD.compute(SqlNewHadoopRDD.scala:66) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:277) > This does not happen in every case, ie. some queries execute fine, and it is > unclear why. > Using just "spark-sql" the query executes fine as well and thus the issue > seems to rely in the communication with Yarn. Also the query executes fine > (with yarn) in spark-shell. -- 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