[jira] [Commented] (SPARK-5741) Support the path contains comma in HiveContext

2015-08-27 Thread koert kuipers (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717235#comment-14717235
 ] 

koert kuipers commented on SPARK-5741:
--

i am reading avro and csv mostly. but we try to support multiple inputs across 
a wide range of formats (currently avro, csv, json, and parquet).

i realize parquet supports it, but it does so by explicitly working around the 
general infrastructure.

i am sympathetic to the idea of no longer doing string munging, but that poses 
some challenges since the main vehicle to carry this information is a 
Map[String, String] (DataFrameReader.extraOptions).

if we could come up with a general way to do this that does not involve string 
munging, i am happy to work on it. the ideal api in my view would be something 
like:
sqlContext.read.format(...).paths(a, b)

alternatively this could be expressed as a union operation of many dataframes, 
but i do not have the knowledge of the relevant code to understand if that is 
feasible, scalable and will support predicate pushdown and such. but if that 
works then i have no need for multiple inputs in DataFrameReader...

from what i know from other projects such as scalding, i think its is a very 
common request to be able to support multiple paths, and you would exclude a 
significant userbase by not supporting it. but thats just a guess...

 Support the path contains comma in HiveContext
 --

 Key: SPARK-5741
 URL: https://issues.apache.org/jira/browse/SPARK-5741
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.2.0
Reporter: Yadong Qi
Assignee: Yadong Qi
 Fix For: 1.3.0


 When run ```select * from nzhang_part where hr = 'file,';```, it throws 
 exception ```java.lang.IllegalArgumentException: Can not create a Path from 
 an empty string```. Because the path of hdfs contains comma, and 
 FileInputFormat.setInputPaths will split path by comma.
 ###
 SQL
 ###
 set hive.merge.mapfiles=true; 
 set hive.merge.mapredfiles=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 create table nzhang_part like srcpart;
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select 
 key, value, hr from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select 
 key, value from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr)  
 select * from (
 select key, value, hr from srcpart where ds='2008-04-08'
 union all
 select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
 select * from nzhang_part where hr = 'file,';
 ###
 Error log
 ###
 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part 
 where hr = 'file,']
 java.lang.IllegalArgumentException: Can not create a Path from an empty string
 at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
 at org.apache.hadoop.fs.Path.init(Path.java:135)
 at 
 org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
 at 
 org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at scala.Option.map(Option.scala:145)
 at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
 at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
  

[jira] [Commented] (SPARK-5741) Support the path contains comma in HiveContext

2015-08-27 Thread koert kuipers (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717053#comment-14717053
 ] 

koert kuipers commented on SPARK-5741:
--

i realize i am late to the party but...

by doing this you are losing a very important functionality: passing in 
multiple input paths comma separated. globs only cover a very limited subset of 
what you can do with multiple paths. for example selecting partitions (by  day) 
for the last 30 days cannot be expressed with a glob.

so you are giving up major functionality just to be able to pass in a character 
that people would generally advice should not be part of a filename anyhow? 
doesnt sound like a good idea to me.

 Support the path contains comma in HiveContext
 --

 Key: SPARK-5741
 URL: https://issues.apache.org/jira/browse/SPARK-5741
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.2.0
Reporter: Yadong Qi
Assignee: Yadong Qi
 Fix For: 1.3.0


 When run ```select * from nzhang_part where hr = 'file,';```, it throws 
 exception ```java.lang.IllegalArgumentException: Can not create a Path from 
 an empty string```. Because the path of hdfs contains comma, and 
 FileInputFormat.setInputPaths will split path by comma.
 ###
 SQL
 ###
 set hive.merge.mapfiles=true; 
 set hive.merge.mapredfiles=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 create table nzhang_part like srcpart;
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select 
 key, value, hr from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select 
 key, value from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr)  
 select * from (
 select key, value, hr from srcpart where ds='2008-04-08'
 union all
 select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
 select * from nzhang_part where hr = 'file,';
 ###
 Error log
 ###
 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part 
 where hr = 'file,']
 java.lang.IllegalArgumentException: Can not create a Path from an empty string
 at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
 at org.apache.hadoop.fs.Path.init(Path.java:135)
 at 
 org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
 at 
 org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at scala.Option.map(Option.scala:145)
 at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
 at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at 

[jira] [Commented] (SPARK-5741) Support the path contains comma in HiveContext

2015-08-27 Thread Michael Armbrust (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717165#comment-14717165
 ] 

Michael Armbrust commented on SPARK-5741:
-

What format are you trying to read?  There [are still ways to read more than 
one 
file|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala#L258],
 they just don't rely on brittle string munging anymore.

 Support the path contains comma in HiveContext
 --

 Key: SPARK-5741
 URL: https://issues.apache.org/jira/browse/SPARK-5741
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.2.0
Reporter: Yadong Qi
Assignee: Yadong Qi
 Fix For: 1.3.0


 When run ```select * from nzhang_part where hr = 'file,';```, it throws 
 exception ```java.lang.IllegalArgumentException: Can not create a Path from 
 an empty string```. Because the path of hdfs contains comma, and 
 FileInputFormat.setInputPaths will split path by comma.
 ###
 SQL
 ###
 set hive.merge.mapfiles=true; 
 set hive.merge.mapredfiles=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 create table nzhang_part like srcpart;
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select 
 key, value, hr from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select 
 key, value from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr)  
 select * from (
 select key, value, hr from srcpart where ds='2008-04-08'
 union all
 select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
 select * from nzhang_part where hr = 'file,';
 ###
 Error log
 ###
 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part 
 where hr = 'file,']
 java.lang.IllegalArgumentException: Can not create a Path from an empty string
 at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
 at org.apache.hadoop.fs.Path.init(Path.java:135)
 at 
 org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
 at 
 org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at scala.Option.map(Option.scala:145)
 at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
 at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at scala.collection.immutable.List.foreach(List.scala:318)
 at 
 scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
 at scala.collection.AbstractTraversable.map(Traversable.scala:105)
 at org.apache.spark.rdd.UnionRDD.getPartitions(UnionRDD.scala:66)
  

[jira] [Commented] (SPARK-5741) Support the path contains comma in HiveContext

2015-08-27 Thread Michael Armbrust (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717258#comment-14717258
 ] 

Michael Armbrust commented on SPARK-5741:
-

It was originally just parquet that would support more than one file, but now 
all HadoopFSRelations should. (which covers all but CSV, and we should upgrade 
that library too)  I would be in favor of generalizing this support for at 
least these sources given the following constraints:

 - We must keep source/binary compatibility.
 - We should give good errors when the source does not support this feature.
 - For consistency, I'd prefer if we can just add a {{load(path: String*)}} 
(but I'm not sure if this is possible given the above).
 - {{paths(path: *)}} is okay, but I think I'd prefer if it was not the 
terminal operator.

 Support the path contains comma in HiveContext
 --

 Key: SPARK-5741
 URL: https://issues.apache.org/jira/browse/SPARK-5741
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.2.0
Reporter: Yadong Qi
Assignee: Yadong Qi
 Fix For: 1.3.0


 When run ```select * from nzhang_part where hr = 'file,';```, it throws 
 exception ```java.lang.IllegalArgumentException: Can not create a Path from 
 an empty string```. Because the path of hdfs contains comma, and 
 FileInputFormat.setInputPaths will split path by comma.
 ###
 SQL
 ###
 set hive.merge.mapfiles=true; 
 set hive.merge.mapredfiles=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 create table nzhang_part like srcpart;
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select 
 key, value, hr from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select 
 key, value from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr)  
 select * from (
 select key, value, hr from srcpart where ds='2008-04-08'
 union all
 select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
 select * from nzhang_part where hr = 'file,';
 ###
 Error log
 ###
 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part 
 where hr = 'file,']
 java.lang.IllegalArgumentException: Can not create a Path from an empty string
 at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
 at org.apache.hadoop.fs.Path.init(Path.java:135)
 at 
 org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
 at 
 org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at scala.Option.map(Option.scala:145)
 at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
 at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at 
 

[jira] [Commented] (SPARK-5741) Support the path contains comma in HiveContext

2015-08-27 Thread koert kuipers (JIRA)

[ 
https://issues.apache.org/jira/browse/SPARK-5741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14717450#comment-14717450
 ] 

koert kuipers commented on SPARK-5741:
--

given the requirement of source/binary compatibility i do not think it can be 
done without some kind of string munging.

however the string munging could be restricted to a separate variable, set with 
paths(path: *) so path does not get polluted. this variable would be 
exclusively for HadoopFsRelationProvider, and an error thrown in 
ResolvedDataSource if any other RelationProvider is used. also an error would 
be thrown if path and paths are both set.

does this sound reasonable? if not i will keep looking for other solutions


 Support the path contains comma in HiveContext
 --

 Key: SPARK-5741
 URL: https://issues.apache.org/jira/browse/SPARK-5741
 Project: Spark
  Issue Type: Bug
  Components: SQL
Affects Versions: 1.2.0
Reporter: Yadong Qi
Assignee: Yadong Qi
 Fix For: 1.3.0


 When run ```select * from nzhang_part where hr = 'file,';```, it throws 
 exception ```java.lang.IllegalArgumentException: Can not create a Path from 
 an empty string```. Because the path of hdfs contains comma, and 
 FileInputFormat.setInputPaths will split path by comma.
 ###
 SQL
 ###
 set hive.merge.mapfiles=true; 
 set hive.merge.mapredfiles=true;
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 set hive.exec.dynamic.partition=true;
 set hive.exec.dynamic.partition.mode=nonstrict;
 create table nzhang_part like srcpart;
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select 
 key, value, hr from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select 
 key, value from srcpart where ds='2008-04-08';
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr)  
 select * from (
 select key, value, hr from srcpart where ds='2008-04-08'
 union all
 select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;
 select * from nzhang_part where hr = 'file,';
 ###
 Error log
 ###
 15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part 
 where hr = 'file,']
 java.lang.IllegalArgumentException: Can not create a Path from an empty string
 at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
 at org.apache.hadoop.fs.Path.init(Path.java:135)
 at 
 org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
 at 
 org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at 
 org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
 at scala.Option.map(Option.scala:145)
 at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
 at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at 
 org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:223)
 at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:221)
 at scala.Option.getOrElse(Option.scala:120)
 at org.apache.spark.rdd.RDD.partitions(RDD.scala:221)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:66)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at 
 scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
 at