-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-5741][SQL] Support the path contains comma in HiveContext #4532
Conversation
Can one of the admins verify this patch? |
ok to test. |
@@ -248,7 +249,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { | |||
* instantiate a HadoopRDD. | |||
*/ | |||
def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { | |||
FileInputFormat.setInputPaths(jobConf, path) | |||
jobConf.set("mapred.input.dir", StringUtils.escapeString(path.toString())) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Instead of setting the conf using the key, can we still use FileInputFormat.setInputPaths
? Like
FileInputFormat.setInputPaths(jobConf, StringUtils.escapeString(path))
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can't, for examples "hdfs://x.x.x.x:9000/user/hive/warehouse/nzhang_part/ds=2010-08-15/hr=file," is will be splited into "hdfs://x.x.x.x:9000/user/hive/warehouse/nzhang_part/ds=2010-08-15/hr=file" and "" by FileInputFormat.getPathStrings, "" will be checked by Path.checkPathArg and
if( path.length() == 0 ) {
throw new IllegalArgumentException("Can not create a Path from an empty string");
}
you can see
FileInputFormat.setInputPaths -> FileInputFormat.getPathStrings -> Path.checkPathArg
in hadoop for detail.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
o, I see. getPathStrings
does not really care if a comma is escaped or not... Can we use public static void setInputPaths(Job job, Path... inputPaths)
? I think it is better to avoid using set
directly with a string key (using a method seems more robust).
ok to test |
Test build #27292 has started for PR 4532 at commit
|
Test build #27292 has finished for PR 4532 at commit
|
Test PASSed. |
lgtm |
@yhuai Can you review it? |
Test build #28034 has started for PR 4532 at commit
|
Test build #28034 has finished for PR 4532 at commit
|
Test PASSed. |
@yhuai Can you review the code for me? |
LGTM |
Thanks! Merging to master and 1.3. |
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) Author: q00251598 <[email protected]> Closes #4532 from watermen/SPARK-5741 and squashes the following commits: 9758ab1 [q00251598] fix bug 1db1a1c [q00251598] use setInputPaths(Job job, Path... inputPaths) b788a72 [q00251598] change FileInputFormat.setInputPaths to jobConf.set and add test suite (cherry picked from commit 9ce12aa) Signed-off-by: Michael Armbrust <[email protected]>
When run
select * from nzhang_part where hr = 'file,';
, it throws exceptionjava.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
Error Log